From ea24003f3f719ec5c5029f8f1dcbf9ccd10d5bfb Mon Sep 17 00:00:00 2001 From: 0xCipherCoder Date: Fri, 21 Feb 2025 05:01:53 +0000 Subject: [PATCH 01/21] doc:fixed broken link and grammatical errors --- README.md | 2 +- doc/architecture.md | 2 +- doc/flow.md | 4 ++-- integration_tests/README.md | 4 ++-- nft_ingester/doc/backfiller.md | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/README.md b/README.md index ba8e99559..7949c9c8a 100644 --- a/README.md +++ b/README.md @@ -8,7 +8,7 @@ The Metaplex Aura Network has three main components: * **Multi-Program Consistent Indexing across any SVM:** Enhanced Digital Asset Standard (DAS) API for performant indexing across any protocols on Solana and the broader SVM. * **Elastic State Management:** Ability to transition assets between SVM account space and different levels of state compression on demand, increasing flexibility and optimizing asset performance and scalability. -For more background information, please see [our blog](http://www.metaplex.com/posts/aura-network). +For more background information, please see [our blog](https://www.metaplex.com/blog/articles/aura-announcement). ## Aura Infrastructure diff --git a/doc/architecture.md b/doc/architecture.md index c3ae1ec57..719cb8576 100644 --- a/doc/architecture.md +++ b/doc/architecture.md @@ -48,7 +48,7 @@ The system is composed of several specialized binaries, each performing a specif * Fetches and persists finalized slot data to the slots RocksDB database. * Provides a reliable source of finalized data for backfilling and validation. * Can fetch data from either BigTable (optional) or an RPC source. - * Handles retries and error handling, ensure the slots data is consistent and consicuential - that is the slot number is always increasing and never skipped (although it still may miss some slots due to the eventual consistency of bigtable). + * Handles retries and error handling, ensuring the slots data is consistent and sequential - that is the slot number is always increasing and never skipped (although it still may miss some slots due to the eventual consistency of bigtable). * **Key Dependencies**: RocksDB, BigTable (optional), Solana RPC. 4. **`backfill`** (`nft_ingester/src/bin/backfill/main.rs`): diff --git a/doc/flow.md b/doc/flow.md index e4525bc5b..b1e385355 100644 --- a/doc/flow.md +++ b/doc/flow.md @@ -39,6 +39,6 @@ There is also the backfill mechanism that is used to load a historical data (met Search data part: 7. A client (end user or another service) makes a call to our JSON RPC endpoint specifying field he wants to search by. -8. The server first goes to the PostgreSQL inxed to find an ID of the required record. -9. Using the ID, the server fetches required metadata from the RocksDB and returns to the client. +8. The server first queries the PostgreSQL index to find an ID of the required record. +9. Using the ID, the server fetches required metadata from the RocksDB and returns it to the client. diff --git a/integration_tests/README.md b/integration_tests/README.md index e7ec1a4a5..336d0adce 100644 --- a/integration_tests/README.md +++ b/integration_tests/README.md @@ -20,10 +20,10 @@ export DEVNET_RPC_URL=... export MAINNET_RPC_URL=... ``` -Afterwards, you can simply run the following command to run tests: +Afterward, you can simply run the following command to run tests: ```cargo test``` -The tests run migrations from fresh and populates the data. You don't need to any other setup. +The tests run migrations from fresh and populate the data. You don't need to do any other setup. ## How do tests work? diff --git a/nft_ingester/doc/backfiller.md b/nft_ingester/doc/backfiller.md index 2e6e4131e..388ae8279 100644 --- a/nft_ingester/doc/backfiller.md +++ b/nft_ingester/doc/backfiller.md @@ -84,7 +84,7 @@ The producer is the **BackfillSource** (BigTable or RPC). From the BubblegumSlots Rocks CF, slots are extracted, and then the block is downloaded with the help of the **BackfillSource**. -Once the block is downloaded and saved, the slot is dropped from the BubblegumSlots CF and also this slot is added to the IngestableSlots CF so the next worker could parse it. +Once the block is downloaded and saved, the slot is dropped from the BubblegumSlots CF and also this slot is added to the IngestableSlots CF so the next worker can parse it. ### Perpetual Block Ingestion @@ -92,6 +92,6 @@ The consumer is the **DirectBlockParser**, which is a struct with a Bubblegum tr The producer is RocksDB. -The **IngestableSlotGetter** returns slots from the IngestableSlots CF, then blocks are extracted from the Rocks. +The **IngestableSlotGetter** returns slots from the IngestableSlots CF, and then blocks are extracted from the Rocks. Once a block is received, it’s parsed, and the slot is dropped from the IngestableSlots CF. \ No newline at end of file From 62b6d94bed0d3f1c552025ec468253b922889f49 Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Tue, 25 Feb 2025 14:03:40 +0200 Subject: [PATCH 02/21] feat: rework data formats used in bubblegum transaction processing (#422) * feat: a direct fetching from bigtable with a test * feat: rework slots processing, simplify data types used for bubblegum * feat!: use bincode to serialize/deserialize raw blocks * feat(rocksdb): open migration db with deprecated CF descriptor * feat(rocksdb): improve raw blocks migration script * feat(rocksdb): add migration readme, optimize migration memory skip-checks: true * chore: improve slot persister performance, clean up deps --------- Co-authored-by: Stanislav Cherviakov --- Cargo.lock | 5 +- Cargo.toml | 1 + backfill_rpc/src/block_producer.rs | 26 ++- entities/src/lib.rs | 1 + entities/src/models.rs | 92 ++++++++- entities/src/transaction_converters.rs | 104 ++++++++++ grpc/Cargo.toml | 2 +- grpc/proto/gap_filler.proto | 3 +- grpc/src/client.rs | 11 +- grpc/src/error/mod.rs | 10 +- grpc/src/gapfiller.rs | 3 +- grpc/src/mapper.rs | 4 +- interface/src/signature_persistence.rs | 8 +- nft_ingester/Cargo.toml | 2 +- nft_ingester/src/backfiller.rs | 48 +---- nft_ingester/src/bin/backfill/main.rs | 2 +- nft_ingester/src/bin/slot_persister/main.rs | 193 +++++++++++++----- nft_ingester/src/plerkle.rs | 15 +- .../bubblegum_updates_processor.rs | 65 +++--- nft_ingester/src/transaction_ingester.rs | 9 +- nft_ingester/tests/clean_forks_test.rs | 77 +++---- nft_ingester/tests/gapfiller_tests.rs | 12 +- rocks-db/Cargo.toml | 4 + rocks-db/src/bin/fork_detector/main.rs | 2 +- rocks-db/src/bin/migrate_slots_db/README.md | 33 +++ rocks-db/src/bin/migrate_slots_db/main.rs | 143 +++++++++++++ .../clients/raw_blocks_streaming_client.rs | 5 +- rocks-db/src/column.rs | 2 +- rocks-db/src/columns/raw_block.rs | 23 ++- rocks-db/src/fork_cleaner.rs | 8 +- rocks-db/src/lib.rs | 13 +- rocks-db/src/transaction.rs | 4 +- .../tests/asset_streaming_client_tests.rs | 12 +- rocks-db/tests/raw_block_tests.rs | 120 ++++++++++- usecase/Cargo.toml | 4 +- usecase/src/bigtable.rs | 137 +++++++++++-- 36 files changed, 913 insertions(+), 290 deletions(-) create mode 100644 entities/src/transaction_converters.rs create mode 100644 rocks-db/src/bin/migrate_slots_db/README.md create mode 100644 rocks-db/src/bin/migrate_slots_db/main.rs diff --git a/Cargo.lock b/Cargo.lock index 008330066..3284c5272 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3314,12 +3314,12 @@ name = "grpc" version = "0.1.0" dependencies = [ "async-trait", + "bincode", "entities", "futures", "interface", "mockall 0.13.0", "prost 0.12.6", - "serde_cbor", "solana-sdk", "solana-transaction-status", "thiserror", @@ -6528,6 +6528,7 @@ dependencies = [ "bincode", "bubblegum-batch-sdk", "chrono", + "clap 4.5.26", "criterion", "csv", "entities", @@ -10379,6 +10380,7 @@ dependencies = [ "mockall 0.13.0", "mpl-bubblegum", "plerkle_serialization", + "prost 0.11.9", "rand 0.8.5", "serde", "serde_derive", @@ -10388,6 +10390,7 @@ dependencies = [ "solana-program", "solana-sdk", "solana-storage-bigtable", + "solana-storage-proto", "solana-transaction-status", "spl-account-compression 0.4.2", "spl-concurrent-merkle-tree 0.4.1", diff --git a/Cargo.toml b/Cargo.toml index e8de9b785..b43cf6bd3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -153,6 +153,7 @@ spl-concurrent-merkle-tree = { version="0.4.1" } spl-account-compression = { version="0.4.2", features = ["no-entrypoint"] } solana-storage-bigtable = "~1.18.11" solana-bigtable-connection = "1.10.31" +solana-storage-proto = "~1.18.11" spl-token = { version = "4.0.0", features = ["no-entrypoint"] } spl-token-2022 = { version = "1.0", features = ["no-entrypoint"] } spl-pod = "0.1.0" diff --git a/backfill_rpc/src/block_producer.rs b/backfill_rpc/src/block_producer.rs index 20f6fe031..77aef82fb 100644 --- a/backfill_rpc/src/block_producer.rs +++ b/backfill_rpc/src/block_producer.rs @@ -1,10 +1,14 @@ use std::{sync::Arc, time::Duration}; use async_trait::async_trait; +use entities::{ + models::{RawBlockWithTransactions, TransactionInfo}, + transaction_converters::decode_encoded_transaction_with_status_meta, +}; use interface::{error::StorageError, signature_persistence::BlockProducer}; use solana_client::rpc_config::RpcBlockConfig; use solana_sdk::commitment_config::{CommitmentConfig, CommitmentLevel}; -use solana_transaction_status::{TransactionDetails, UiConfirmedBlock}; +use solana_transaction_status::TransactionDetails; use tracing::error; use usecase::bigtable::is_bubblegum_transaction_encoded; @@ -18,7 +22,7 @@ impl BlockProducer for BackfillRPC { &self, slot: u64, _backup_provider: Option>, - ) -> Result { + ) -> Result { let mut counter = MAX_RPC_RETRIES; loop { @@ -52,8 +56,24 @@ impl BlockProducer for BackfillRPC { if let Some(ref mut txs) = encoded_block.transactions { txs.retain(is_bubblegum_transaction_encoded); } + let raw_block = RawBlockWithTransactions { + blockhash: encoded_block.blockhash, + previous_blockhash: encoded_block.previous_blockhash, + parent_slot: encoded_block.parent_slot, + transactions: encoded_block + .transactions + .unwrap_or_default() + .into_iter() + .filter_map(|t| { + decode_encoded_transaction_with_status_meta(t).and_then(|t| { + TransactionInfo::from_transaction_with_status_meta_and_slot(t, slot) + }) + }) + .collect(), + block_time: encoded_block.block_time.and_then(|t| t.try_into().ok()), + }; - return Ok(encoded_block); + return Ok(raw_block); } } } diff --git a/entities/src/lib.rs b/entities/src/lib.rs index 557dd5b0d..59f96365f 100644 --- a/entities/src/lib.rs +++ b/entities/src/lib.rs @@ -2,3 +2,4 @@ pub mod api_req_params; pub mod enums; pub mod models; pub mod schedule; +pub mod transaction_converters; diff --git a/entities/src/models.rs b/entities/src/models.rs index 183fc7fca..3168ce52b 100644 --- a/entities/src/models.rs +++ b/entities/src/models.rs @@ -12,7 +12,13 @@ use mpl_token_metadata::accounts::Metadata; use schemars::JsonSchema; use serde::{Deserialize, Serialize}; use sha2::{Digest, Sha256}; -use solana_sdk::{hash::Hash, pubkey::Pubkey, signature::Signature}; +use solana_sdk::{ + hash::Hash, instruction::CompiledInstruction, message::v0::LoadedAddresses, pubkey::Pubkey, + signature::Signature, +}; +use solana_transaction_status::{ + EncodedConfirmedBlock, InnerInstructions, TransactionWithStatusMeta, +}; use sqlx::types::chrono; use crate::enums::{ @@ -418,12 +424,92 @@ pub struct Task { pub ofd_status: TaskStatus, } -#[derive(Serialize, Deserialize, Debug, Clone)] -pub struct RawBlock { +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq)] +pub struct RawBlockDeprecated { pub slot: u64, pub block: solana_transaction_status::UiConfirmedBlock, } +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq)] +pub struct RawBlock { + pub slot: u64, + pub block: RawBlockWithTransactions, +} + +impl From for RawBlock { + fn from(value: RawBlockDeprecated) -> Self { + let encoded: EncodedConfirmedBlock = value.block.into(); + + Self { + slot: value.slot, + block: RawBlockWithTransactions { + blockhash: encoded.blockhash, + previous_blockhash: encoded.previous_blockhash, + parent_slot: encoded.parent_slot, + block_time: encoded.block_time.and_then(|t| t.try_into().ok()), + transactions: encoded + .transactions + .into_iter() + .filter_map(|t| { + crate::transaction_converters::decode_encoded_transaction_with_status_meta( + t, + ) + .and_then(|tx| { + TransactionInfo::from_transaction_with_status_meta_and_slot( + tx, value.slot, + ) + }) + }) + .collect(), + }, + } + } +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +pub struct RawBlockWithTransactions { + pub blockhash: String, + pub previous_blockhash: String, + pub parent_slot: u64, + pub block_time: Option, + pub transactions: Vec, +} + +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +pub struct TransactionInfo { + pub slot: u64, + pub signature: Signature, + pub account_keys: Vec, + pub message_instructions: Vec, + pub meta_inner_instructions: Vec, +} + +impl TransactionInfo { + pub fn from_transaction_with_status_meta_and_slot( + tx: TransactionWithStatusMeta, + slot: u64, + ) -> Option { + let meta = tx.get_status_meta()?; + let tx = tx.get_transaction(); + let atl_keys = tx.message.address_table_lookups(); + let inner_instructions = meta.inner_instructions.unwrap_or_default(); + let mut account_keys = tx.message.static_account_keys().to_vec(); + if atl_keys.is_some() { + let LoadedAddresses { writable, readonly } = meta.loaded_addresses; + account_keys.extend(writable); + account_keys.extend(readonly); + } + + Some(Self { + slot, + signature: tx.signatures[0], + account_keys, + message_instructions: tx.message.instructions().to_vec(), + meta_inner_instructions: inner_instructions, + }) + } +} + #[derive(Debug, Clone)] pub struct JsonDownloadTask { pub metadata_url: String, diff --git a/entities/src/transaction_converters.rs b/entities/src/transaction_converters.rs new file mode 100644 index 000000000..db8c61b97 --- /dev/null +++ b/entities/src/transaction_converters.rs @@ -0,0 +1,104 @@ +use std::str::FromStr; + +use base64::Engine; +use solana_sdk::{ + bs58, instruction::CompiledInstruction, message::v0::LoadedAddresses, pubkey::Pubkey, + transaction_context::TransactionReturnData, +}; +use solana_transaction_status::{ + option_serializer::OptionSerializer, EncodedTransactionWithStatusMeta, InnerInstruction, + InnerInstructions, TransactionStatusMeta, TransactionTokenBalance, TransactionWithStatusMeta, + UiInstruction, UiTransactionTokenBalance, VersionedTransactionWithStatusMeta, +}; + +pub fn decode_encoded_transaction_with_status_meta( + t: EncodedTransactionWithStatusMeta, +) -> Option { + fn get(s: OptionSerializer) -> Option { + s.into() + } + match t.meta { + Some(m) => { + let inner_instructions = get(m.inner_instructions).map(|ii| { + ii.into_iter() + .map(|ii| InnerInstructions { + index: ii.index, + instructions: ii + .instructions + .into_iter() + .filter_map(|ui| { + if let UiInstruction::Compiled(i) = ui { + Some(InnerInstruction { + instruction: CompiledInstruction { + program_id_index: i.program_id_index, + accounts: i.accounts, + data: bs58::decode(i.data).into_vec().ok()?, + }, + stack_height: i.stack_height, + }) + } else { + None + } + }) + .collect(), + }) + .collect() + }); + let meta = TransactionStatusMeta { + status: m.status, + fee: m.fee, + pre_balances: m.pre_balances, + post_balances: m.post_balances, + inner_instructions, + log_messages: get(m.log_messages), + pre_token_balances: get(m.pre_token_balances) + .map(decode_transaction_token_balances), + post_token_balances: get(m.post_token_balances) + .map(decode_transaction_token_balances), + rewards: m.rewards.into(), + loaded_addresses: get(m.loaded_addresses) + .map(|ula| LoadedAddresses { + writable: ula + .writable + .into_iter() + .filter_map(|a| Pubkey::from_str(&a).ok()) + .collect(), + readonly: ula + .readonly + .into_iter() + .filter_map(|a| Pubkey::from_str(&a).ok()) + .collect(), + }) + .unwrap_or_default(), + return_data: get(m.return_data).and_then(|urd| { + Some(TransactionReturnData { + program_id: Pubkey::from_str(&urd.program_id).ok()?, + data: base64::prelude::BASE64_STANDARD.decode(urd.data.0).ok()?, + }) + }), + compute_units_consumed: m.compute_units_consumed.into(), + }; + let versioned_txn_with_status_meta = + VersionedTransactionWithStatusMeta { transaction: t.transaction.decode()?, meta }; + let transaction_with_status_meta = + TransactionWithStatusMeta::Complete(versioned_txn_with_status_meta); + Some(transaction_with_status_meta) + }, + None => None, + } +} + +fn decode_transaction_token_balances( + ui_transaction_token_balances: Vec, +) -> Vec { + ui_transaction_token_balances + .into_iter() + .map(|uttb| TransactionTokenBalance { + account_index: uttb.account_index, + mint: uttb.mint, + ui_token_amount: uttb.ui_token_amount, + owner: Option::::from(uttb.owner).unwrap_or_default(), + program_id: Option::::from(uttb.program_id).unwrap_or_default(), + }) + .collect() +} diff --git a/grpc/Cargo.toml b/grpc/Cargo.toml index 939896cf1..6354b88fb 100644 --- a/grpc/Cargo.toml +++ b/grpc/Cargo.toml @@ -6,6 +6,7 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] +bincode = { workspace = true } tokio = { workspace = true } tonic = { workspace = true } prost = { workspace = true } @@ -16,7 +17,6 @@ entities = { path = "../entities" } async-trait = { workspace = true } thiserror = { workspace = true } solana-transaction-status = { workspace = true } -serde_cbor = { workspace = true } tracing = { workspace = true } [dev-dependencies] diff --git a/grpc/proto/gap_filler.proto b/grpc/proto/gap_filler.proto index 6e1ce2166..6f6ad7a65 100644 --- a/grpc/proto/gap_filler.proto +++ b/grpc/proto/gap_filler.proto @@ -171,8 +171,7 @@ message AssetDetails { bytes owner_record_pubkey = 45; } -// This field is a cbor encoded object of RawBlock type depending on -// a "~1.17" version of the solana-transaction-status package +// This field is a bincode encoded object of RawBlock type defined in `entities`. message RawBlock { bytes block = 1; } diff --git a/grpc/src/client.rs b/grpc/src/client.rs index 7fe0978af..707bcc7e9 100644 --- a/grpc/src/client.rs +++ b/grpc/src/client.rs @@ -1,6 +1,7 @@ use std::{str::FromStr, sync::Arc}; use async_trait::async_trait; +use entities::models::{RawBlock, RawBlockWithTransactions}; use futures::StreamExt; use interface::{ asset_streaming_and_discovery::{ @@ -94,7 +95,7 @@ impl BlockProducer for Client { &self, slot: u64, backup_provider: Option>, - ) -> Result { + ) -> Result { if let Ok(block) = self .inner .clone() @@ -102,11 +103,9 @@ impl BlockProducer for Client { .await .map_err(|e| StorageError::Common(e.to_string())) .and_then(|response| { - serde_cbor::from_slice::( - response.into_inner().block.as_slice(), - ) - .map_err(|e| StorageError::Common(e.to_string())) - .map(|raw_block| raw_block.block) + bincode::deserialize::(response.into_inner().block.as_slice()) + .map_err(|e| StorageError::Common(e.to_string())) + .map(|raw_block| raw_block.block) }) { return Ok(block); diff --git a/grpc/src/error/mod.rs b/grpc/src/error/mod.rs index c4bc1834e..b0f7e71bb 100644 --- a/grpc/src/error/mod.rs +++ b/grpc/src/error/mod.rs @@ -13,8 +13,8 @@ pub enum GrpcError { UriCreate(String), #[error("TonicTransport: {0}")] TonicTransport(String), - #[error("Cbor: {0}")] - Cbor(String), + #[error("Bincode error: {0}")] + Bincode(String), } impl From for GrpcError { fn from(value: Error) -> Self { @@ -22,8 +22,8 @@ impl From for GrpcError { } } -impl From for GrpcError { - fn from(value: serde_cbor::error::Error) -> Self { - Self::Cbor(value.to_string()) +impl From for GrpcError { + fn from(value: bincode::Error) -> Self { + Self::Bincode(value.to_string()) } } diff --git a/grpc/src/gapfiller.rs b/grpc/src/gapfiller.rs index 18cb4cf9b..57d2e8594 100644 --- a/grpc/src/gapfiller.rs +++ b/grpc/src/gapfiller.rs @@ -197,8 +197,7 @@ pub struct AssetDetails { #[prost(bytes = "vec", tag = "45")] pub owner_record_pubkey: ::prost::alloc::vec::Vec, } -/// This field is a cbor encoded object of RawBlock type depending on -/// a "~1.17" version of the solana-transaction-status package +/// This field is a bincode encoded object of RawBlock type defined in `entities`. #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct RawBlock { diff --git a/grpc/src/mapper.rs b/grpc/src/mapper.rs index 083775c00..16ac432a5 100644 --- a/grpc/src/mapper.rs +++ b/grpc/src/mapper.rs @@ -778,7 +778,7 @@ impl TryFrom for RawBlock { type Error = GrpcError; fn try_from(value: entities::models::RawBlock) -> Result { - Ok(Self { block: serde_cbor::to_vec(&value).map_err(Into::::into)? }) + Ok(Self { block: bincode::serialize(&value).map_err(Into::::into)? }) } } @@ -786,7 +786,7 @@ impl TryFrom for entities::models::RawBlock { type Error = GrpcError; fn try_from(value: RawBlock) -> Result { - serde_cbor::from_slice(value.block.as_slice()).map_err(Into::::into) + bincode::deserialize(value.block.as_slice()).map_err(Into::::into) } } diff --git a/interface/src/signature_persistence.rs b/interface/src/signature_persistence.rs index 695601085..f697dfe2b 100644 --- a/interface/src/signature_persistence.rs +++ b/interface/src/signature_persistence.rs @@ -1,7 +1,9 @@ use std::sync::Arc; use async_trait::async_trait; -use entities::models::{BufferedTransaction, BufferedTxWithID, SignatureWithSlot}; +use entities::models::{ + BufferedTransaction, BufferedTxWithID, RawBlockWithTransactions, SignatureWithSlot, +}; use mockall::automock; use solana_sdk::pubkey::Pubkey; @@ -47,7 +49,7 @@ pub trait BlockConsumer: Send + Sync + 'static { async fn consume_block( &self, slot: u64, - block: solana_transaction_status::UiConfirmedBlock, + block: RawBlockWithTransactions, ) -> Result<(), BlockConsumeError>; async fn already_processed_slot(&self, slot: u64) -> Result; } @@ -59,5 +61,5 @@ pub trait BlockProducer: Send + Sync + 'static { &self, slot: u64, backup_provider: Option>, - ) -> Result; + ) -> Result; } diff --git a/nft_ingester/Cargo.toml b/nft_ingester/Cargo.toml index 8176922fd..73205f5b9 100644 --- a/nft_ingester/Cargo.toml +++ b/nft_ingester/Cargo.toml @@ -80,11 +80,11 @@ tokio-retry = { workspace = true } axum = { workspace = true } rocksdb = { workspace = true } num-bigint = "0.4" +tracing-test = { workspace = true } [dev-dependencies] setup = { path = "../tests/setup" } testcontainers = { workspace = true } -tracing-test = { workspace = true } criterion = { workspace = true } zip-extract = { workspace = true } assertables = "7" diff --git a/nft_ingester/src/backfiller.rs b/nft_ingester/src/backfiller.rs index f6d3a6ee2..640994fe9 100644 --- a/nft_ingester/src/backfiller.rs +++ b/nft_ingester/src/backfiller.rs @@ -2,15 +2,13 @@ use std::{collections::HashMap, sync::Arc, time}; use async_trait::async_trait; use backfill_rpc::rpc::BackfillRPC; -use entities::models::{BufferedTransaction, RawBlock}; -use flatbuffers::FlatBufferBuilder; +use entities::models::{RawBlock, RawBlockWithTransactions, TransactionInfo}; use interface::{ error::{BlockConsumeError, StorageError, UsecaseError}, signature_persistence::{BlockConsumer, BlockProducer}, slots_dumper::{SlotGetter, SlotsDumper}, }; use metrics_utils::BackfillerMetricsConfig; -use plerkle_serialization::serializer::seralize_encoded_transaction_with_status; use rocks_db::{ column::TypedColumn, columns::bubblegum_slots::ForceReingestableSlots, @@ -18,14 +16,11 @@ use rocks_db::{ SlotStorage, Storage, }; use solana_program::pubkey::Pubkey; -use solana_transaction_status::{ - EncodedConfirmedTransactionWithStatusMeta, EncodedTransactionWithStatusMeta, UiConfirmedBlock, -}; use tokio::time::Duration; use tokio_util::sync::CancellationToken; use tracing::{error, info}; use usecase::{ - bigtable::{is_bubblegum_transaction_encoded, BigTableClient}, + bigtable::{is_bubblegum_transaction_from_info, BigTableClient}, slots_collector::SlotsGetter, }; @@ -95,7 +90,7 @@ impl BlockProducer for BackfillSource { &self, slot: u64, backup_provider: Option>, - ) -> Result { + ) -> Result { match self { BackfillSource::Bigtable(bigtable) => bigtable.get_block(slot, backup_provider).await, BackfillSource::Rpc(rpc) => rpc.get_block(slot, backup_provider).await, @@ -194,7 +189,7 @@ where if let Some((key, raw_block_data)) = it.item() { let slot = RawBlock::decode_key(key.to_vec())?; // Process the slot - let raw_block: RawBlock = match serde_cbor::from_slice(raw_block_data) { + let raw_block: RawBlock = match RawBlock::decode(raw_block_data) { Ok(rb) => rb, Err(e) => { error!("Failed to decode the value for slot {}: {}", slot, e); @@ -207,7 +202,7 @@ where } if let Some(block_time) = block_time { let dur = time::SystemTime::now() - .duration_since(time::UNIX_EPOCH + Duration::from_secs(block_time as u64)) + .duration_since(time::UNIX_EPOCH + Duration::from_secs(block_time)) .unwrap_or_default() .as_millis() as f64; metrics.set_slot_delay_time("raw_slot_backfilled", dur); @@ -227,36 +222,18 @@ where async fn consume_block( &self, slot: u64, - block: solana_transaction_status::UiConfirmedBlock, + block: RawBlockWithTransactions, ) -> Result<(), BlockConsumeError> { - if block.transactions.is_none() { + if block.transactions.is_empty() { return Ok(()); } - let txs: Vec = block.transactions.unwrap(); + let txs: Vec = block.transactions; let mut results = Vec::new(); for tx in txs.iter() { - if !is_bubblegum_transaction_encoded(tx) { + if !is_bubblegum_transaction_from_info(tx) { continue; } - let builder = FlatBufferBuilder::new(); - let encoded_tx = tx.clone(); - let tx_wrap = EncodedConfirmedTransactionWithStatusMeta { - transaction: encoded_tx, - slot, - block_time: block.block_time, - }; - - let builder = match seralize_encoded_transaction_with_status(builder, tx_wrap) { - Ok(builder) => builder, - Err(err) => { - error!("Error serializing transaction with plerkle: {}", err); - continue; - }, - }; - - let tx = builder.finished_data().to_vec(); - let tx = BufferedTransaction { transaction: tx, map_flatbuffer: false }; match self .ingester .get_ingest_transaction_results(tx.clone()) @@ -267,10 +244,7 @@ where self.metrics.inc_data_processed("backfiller_tx_processed"); }, Err(e) => { - let signature = - plerkle_serialization::root_as_transaction_info(tx.transaction.as_slice()) - .map(|parsed_tx| parsed_tx.signature().unwrap_or_default()) - .unwrap_or_default(); + let signature = tx.signature; error!("Failed to ingest transaction {}: {}", signature, e); self.metrics.inc_data_processed("backfiller_tx_processed_failed"); }, @@ -382,7 +356,7 @@ where async fn consume_block( &self, slot: u64, - block: solana_transaction_status::UiConfirmedBlock, + block: RawBlockWithTransactions, ) -> Result<(), BlockConsumeError> { self.direct_block_parser.consume_block(slot, block).await } diff --git a/nft_ingester/src/bin/backfill/main.rs b/nft_ingester/src/bin/backfill/main.rs index 2fa463e90..2f03a5c5d 100644 --- a/nft_ingester/src/bin/backfill/main.rs +++ b/nft_ingester/src/bin/backfill/main.rs @@ -208,7 +208,7 @@ async fn main() { } // Process the slot - let raw_block: RawBlock = match serde_cbor::from_slice(&raw_block_data) { + let raw_block: RawBlock = match RawBlock::decode(&raw_block_data) { Ok(rb) => rb, Err(e) => { error!("Failed to decode the value for slot {}: {}", slot, e); diff --git a/nft_ingester/src/bin/slot_persister/main.rs b/nft_ingester/src/bin/slot_persister/main.rs index a741f7cb5..06dc266e9 100644 --- a/nft_ingester/src/bin/slot_persister/main.rs +++ b/nft_ingester/src/bin/slot_persister/main.rs @@ -4,15 +4,23 @@ use backfill_rpc::rpc::BackfillRPC; use clap::Parser; use entities::models::RawBlock; use futures::future::join_all; -use interface::{signature_persistence::BlockProducer, slot_getter::FinalizedSlotGetter}; +use interface::{ + error::StorageError, signature_persistence::BlockProducer, slot_getter::FinalizedSlotGetter, +}; use metrics_utils::{utils::start_metrics, MetricState, MetricsTrait}; use nft_ingester::{backfiller::BackfillSource, inmemory_slots_dumper::InMemorySlotsDumper}; use rocks_db::{column::TypedColumn, SlotStorage}; -use tokio::sync::{broadcast, Semaphore}; +use tokio::{ + sync::{broadcast, Semaphore}, + task::JoinSet, +}; use tokio_retry::{strategy::ExponentialBackoff, RetryIf}; use tokio_util::sync::CancellationToken; use tracing::{debug, error, info, warn}; -use usecase::{bigtable::BigTableClient, slots_collector::SlotsCollector}; +use usecase::{ + bigtable::{get_blocks, BigTableClient}, + slots_collector::SlotsCollector, +}; const MAX_RETRIES: usize = 5; const INITIAL_DELAY_MS: u64 = 100; @@ -338,71 +346,148 @@ async fn process_slots( break; } - let semaphore = Arc::new(Semaphore::new(args.max_concurrency)); - - let fetch_futures = slots_to_fetch.iter().map(|&slot| { - let backfill_source = backfill_source.clone(); - let semaphore = semaphore.clone(); - let shutdown_token = shutdown_token.clone(); - - async move { - let _permit = semaphore.acquire().await; - fetch_block_with_retries(backfill_source, slot, shutdown_token).await - } - }); - - let results = join_all(fetch_futures).await; - let mut new_failed_slots = Vec::new(); - for result in results { - match result { - Ok((slot, raw_block)) => { - successful_blocks.insert(slot, raw_block); - }, - Err((slot, e)) => { - new_failed_slots.push(slot); - error!("Failed to fetch slot {}: {:?}", slot, e); - }, - } + let backfill_source = backfill_source.clone(); + let semaphore = Arc::new(Semaphore::new(args.max_concurrency)); + let shutdown_token = shutdown_token.clone(); + + match &*backfill_source { + // ------------------------------------------------------------------ + // 1) Bigtable path: split the batch into sub-chunks, fetch in parallel + // ------------------------------------------------------------------ + BackfillSource::Bigtable(bigtable_client) => { + let total = slots_to_fetch.len(); + // Force sub_chunk_size to at least 1 + let sub_chunk_size = std::cmp::max(total / args.max_concurrency, 1); + let sub_chunks: Vec<&[u64]> = slots_to_fetch.chunks(sub_chunk_size).collect(); + + info!( + "Bigtable path: Splitting {} slots into {} sub-chunks (max_concurrency={}).", + total, + sub_chunks.len(), + args.max_concurrency + ); + let mut js = JoinSet::new(); + + sub_chunks.into_iter().for_each(|sub_slots| { + let sub_slots = sub_slots.to_vec(); + let bigtable_client = bigtable_client.clone(); + let shutdown_token = shutdown_token.clone(); + js.spawn(async move { + if shutdown_token.is_cancelled() { + error!( + "Failed to fetch sub-chunk of slots (from {:?} to {:?}) due to cancellation", + sub_slots.first(), + sub_slots.last() + ); + (sub_slots.clone(), Err(StorageError::Common("shutdown".to_owned()))) + } else { + ( + sub_slots.clone(), + get_blocks( + &bigtable_client.big_table_inner_client, + sub_slots.as_slice(), + ) + .await, + ) + } + }); + }); + while let Some(result) = js.join_next().await { + match result { + Ok((_, Ok(blocks_map))) => { + for (slot, confirmed_block) in blocks_map { + successful_blocks + .insert(slot, RawBlock { slot, block: confirmed_block }); + } + }, + Ok((sub_slots, Err(e))) => { + error!("Failed to fetch sub-chunk of slots: {}", e); + new_failed_slots.extend(sub_slots); + }, + Err(e) => { + error!("Failed to join a task: {}", e); + new_failed_slots.extend_from_slice(&slots_to_fetch); + }, + } + } + new_failed_slots.sort(); + new_failed_slots.dedup(); + }, + + // --------------------------------------------------------- + // 2) RPC or other: original slot-by-slot concurrency + // --------------------------------------------------------- + _ => { + let fetch_futures = slots_to_fetch.iter().map(|&slot| { + let backfill_source = backfill_source.clone(); + let semaphore = semaphore.clone(); + let shutdown_token = shutdown_token.clone(); + + async move { + let _permit = semaphore.acquire().await; + fetch_block_with_retries(backfill_source, slot, shutdown_token).await + } + }); + + let results = join_all(fetch_futures).await; + for result in results { + match result { + Ok((slot, raw_block)) => { + successful_blocks.insert(slot, raw_block); + }, + Err((slot, e)) => { + error!("Failed to fetch slot {}: {:?}", slot, e); + new_failed_slots.push(slot); + }, + } + } + }, } if new_failed_slots.is_empty() { - // All slots fetched successfully, save to database debug!( - "All slots fetched successfully for current batch. Saving {} slots to RocksDB.", + "All slots fetched in this batch. Attempting to save {} blocks to RocksDB...", successful_blocks.len() ); - if let Err(e) = target_db.raw_blocks_cbor.put_batch(successful_blocks.clone()).await - { - error!("Failed to save blocks to RocksDB: {}", e); - // Handle error or retry saving as needed - batch_retries += 1; - if batch_retries >= MAX_BATCH_RETRIES { - panic!( - "Failed to save batch to RocksDB after {} retries. Discarding batch.", - MAX_BATCH_RETRIES - ); - } else { - warn!( - "Retrying batch save {}/{} after {} ms due to error.", - batch_retries, MAX_BATCH_RETRIES, batch_delay_ms + + let projected_last_slot = successful_blocks.keys().max().copied().unwrap_or(0); + let successful_blocks_len = successful_blocks.len(); + match target_db.raw_blocks.put_batch(std::mem::take(&mut successful_blocks)).await { + Ok(_) => { + info!( + "Successfully saved {} blocks to RocksDB. Last stored slot: {}", + successful_blocks_len, projected_last_slot ); - tokio::time::sleep(Duration::from_millis(batch_delay_ms)).await; - batch_delay_ms *= 2; - } - } else { - // Successfully saved, proceed to next batch - let last_slot = successful_blocks.keys().max().cloned().unwrap_or(0); - info!("Successfully saved batch to RocksDB. Last stored slot: {}", last_slot); - break; + break; // Move on to next chunk of `slots` + }, + Err(e) => { + // DB write failed + error!("Failed to save {} blocks to RocksDB: {}", successful_blocks_len, e); + batch_retries += 1; + if batch_retries >= MAX_BATCH_RETRIES { + panic!( + "Failed to save batch to RocksDB after {} retries. Discarding batch.", + MAX_BATCH_RETRIES + ); + } else { + warn!( + "Retrying batch save {}/{} after {} ms due to error: {}", + batch_retries, MAX_BATCH_RETRIES, batch_delay_ms, e + ); + tokio::time::sleep(Duration::from_millis(batch_delay_ms)).await; + batch_delay_ms *= 2; + } + }, } } else { batch_retries += 1; if batch_retries >= MAX_BATCH_RETRIES { panic!( - "Failed to fetch all slots in batch after {} retries. Discarding batch.", - MAX_BATCH_RETRIES + "Failed to fetch all slots in batch after {} retries. Discarding batch. \ + Slots that failed: {:?}", + MAX_BATCH_RETRIES, new_failed_slots ); } else { warn!( diff --git a/nft_ingester/src/plerkle.rs b/nft_ingester/src/plerkle.rs index 4e46b61aa..e07e13b97 100644 --- a/nft_ingester/src/plerkle.rs +++ b/nft_ingester/src/plerkle.rs @@ -1,7 +1,5 @@ use plerkle_serialization::deserializer::*; -use solana_program::{instruction::CompiledInstruction, pubkey::Pubkey}; -use solana_sdk::signature::Signature; -use solana_transaction_status::InnerInstructions; +use solana_program::pubkey::Pubkey; #[derive(thiserror::Error, Clone, Debug, PartialEq)] pub enum PlerkleDeserializerError { @@ -23,15 +21,6 @@ pub struct AccountInfo { pub data: Vec, } -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct TransactionInfo { - pub slot: u64, - pub signature: Signature, - pub account_keys: Vec, - pub message_instructions: Vec, - pub meta_inner_instructions: Vec, -} - pub struct PlerkleAccountInfo<'a>(pub plerkle_serialization::AccountInfo<'a>); impl TryFrom> for AccountInfo { @@ -55,7 +44,7 @@ impl TryFrom> for AccountInfo { pub struct PlerkleTransactionInfo<'a>(pub plerkle_serialization::TransactionInfo<'a>); -impl<'a> TryFrom> for TransactionInfo { +impl<'a> TryFrom> for entities::models::TransactionInfo { type Error = PlerkleDeserializerError; fn try_from(value: PlerkleTransactionInfo<'a>) -> Result { diff --git a/nft_ingester/src/processors/transaction_based/bubblegum_updates_processor.rs b/nft_ingester/src/processors/transaction_based/bubblegum_updates_processor.rs index 3728be102..449e93f02 100644 --- a/nft_ingester/src/processors/transaction_based/bubblegum_updates_processor.rs +++ b/nft_ingester/src/processors/transaction_based/bubblegum_updates_processor.rs @@ -21,7 +21,7 @@ use entities::{ }, models::{ BatchMintToVerify, BufferedTransaction, ChainDataV1, Creator, SignatureWithSlot, - UpdateVersion, Updated, Uses, + TransactionInfo, UpdateVersion, Updated, Uses, }, }; use lazy_static::lazy_static; @@ -49,8 +49,7 @@ use tracing::{debug, error}; use usecase::save_metrics::result_to_metrics; use crate::{ - error::IngesterError, flatbuffer_mapper::FlatbufferMapper, plerkle, - plerkle::PlerkleTransactionInfo, + error::IngesterError, flatbuffer_mapper::FlatbufferMapper, plerkle::PlerkleTransactionInfo, }; pub const BUFFER_PROCESSING_COUNTER: i32 = 10; @@ -82,9 +81,7 @@ impl BubblegumTxProcessor { } } - pub fn break_transaction( - tx_info: &plerkle::TransactionInfo, - ) -> VecDeque<(IxPair, Option>)> { + pub fn break_transaction(tx_info: &TransactionInfo) -> VecDeque<(IxPair, Option>)> { order_instructions( &KEY_SET, tx_info.account_keys.as_slice(), @@ -102,10 +99,10 @@ impl BubblegumTxProcessor { return Ok(()); } let begin_processing = Instant::now(); - let result = Self::get_process_transaction_results( - data, + let data = Self::parse_transaction_info_from_fb(data, self.transaction_parser.clone())?; + let result = Self::get_handle_transaction_results( self.instruction_parser.clone(), - self.transaction_parser.clone(), + data, self.metrics.clone(), )?; @@ -122,33 +119,6 @@ impl BubblegumTxProcessor { res } - pub fn get_process_transaction_results( - data: BufferedTransaction, - instruction_parser: Arc, - transaction_parser: Arc, - metrics: Arc, - ) -> Result { - let seen_at = Utc::now(); - - let mut transaction_info_bytes = data.transaction.clone(); - - if data.map_flatbuffer { - let tx_update = - utils::flatbuffer::transaction_info_generated::transaction_info::root_as_transaction_info( - &data.transaction, - ).unwrap(); - transaction_info_bytes = - transaction_parser.map_tx_fb_bytes(tx_update, seen_at).unwrap(); - } - let transaction_info = - plerkle_serialization::root_as_transaction_info(transaction_info_bytes.as_slice()) - .unwrap(); - let transaction_info: plerkle::TransactionInfo = - PlerkleTransactionInfo(transaction_info).try_into()?; - - Self::get_handle_transaction_results(instruction_parser, transaction_info, metrics) - } - fn instruction_name_to_string(ix: &InstructionName) -> &'static str { match ix { InstructionName::Unknown => "Unknown", @@ -185,10 +155,31 @@ impl BubblegumTxProcessor { result[..4].copy_from_slice(&bytes); result } + pub fn parse_transaction_info_from_fb( + data: BufferedTransaction, + transaction_parser: Arc, + ) -> Result { + let seen_at = Utc::now(); + let mut transaction_info_bytes = data.transaction.clone(); + + if data.map_flatbuffer { + let tx_update = + utils::flatbuffer::transaction_info_generated::transaction_info::root_as_transaction_info( + &data.transaction, + ).unwrap(); + transaction_info_bytes = + transaction_parser.map_tx_fb_bytes(tx_update, seen_at).unwrap(); + } + let transaction_info = + plerkle_serialization::root_as_transaction_info(transaction_info_bytes.as_slice()) + .unwrap(); + + Ok(PlerkleTransactionInfo(transaction_info).try_into()?) + } pub fn get_handle_transaction_results( instruction_parser: Arc, - tx: plerkle::TransactionInfo, + tx: TransactionInfo, metrics: Arc, ) -> Result { let sig = tx.signature; diff --git a/nft_ingester/src/transaction_ingester.rs b/nft_ingester/src/transaction_ingester.rs index b8d7dc316..58452d61a 100644 --- a/nft_ingester/src/transaction_ingester.rs +++ b/nft_ingester/src/transaction_ingester.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use entities::models::BufferedTransaction; +use entities::models::{BufferedTransaction, TransactionInfo}; use interface::{error::StorageError, signature_persistence::TransactionIngester}; use rocks_db::transaction::{TransactionProcessor, TransactionResult}; use tonic::async_trait; @@ -32,12 +32,11 @@ impl TransactionIngester for BackfillTransactionIngester { impl TransactionProcessor for BackfillTransactionIngester { fn get_ingest_transaction_results( &self, - tx: BufferedTransaction, + tx: TransactionInfo, ) -> Result { - BubblegumTxProcessor::get_process_transaction_results( - tx, + BubblegumTxProcessor::get_handle_transaction_results( self.tx_processor.instruction_parser.clone(), - self.tx_processor.transaction_parser.clone(), + tx, self.tx_processor.metrics.clone(), ) .map_err(|e| StorageError::Common(e.to_string())) diff --git a/nft_ingester/tests/clean_forks_test.rs b/nft_ingester/tests/clean_forks_test.rs index 84798946d..c4fbf23a1 100644 --- a/nft_ingester/tests/clean_forks_test.rs +++ b/nft_ingester/tests/clean_forks_test.rs @@ -1,6 +1,6 @@ use bincode::deserialize; use blockbuster::{instruction::InstructionBundle, programs::bubblegum::BubblegumInstruction}; -use entities::models::{RawBlock, SignatureWithSlot}; +use entities::models::{RawBlock, RawBlockWithTransactions, SignatureWithSlot}; use metrics_utils::MetricState; use mpl_bubblegum::{ types::{BubblegumEventType, LeafSchema, Version}, @@ -31,10 +31,9 @@ async fn test_clean_forks() { str::FromStr, }; - use entities::models::{UpdateVersion, Updated}; + use entities::models::{RawBlockWithTransactions, UpdateVersion, Updated}; use metrics_utils::{utils::start_metrics, MetricsTrait}; use rocks_db::columns::{cl_items::ClItemKey, leaf_signatures::LeafSignature}; - use solana_transaction_status::UiConfirmedBlock; let RocksTestEnvironment { storage, slot_storage, .. } = RocksTestEnvironment::new(&[]); let first_tree_key = @@ -416,100 +415,85 @@ async fn test_clean_forks() { .unwrap(); slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 10000, RawBlock { slot: 10000, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) .await .unwrap(); slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 10001, RawBlock { slot: 10001, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) .await .unwrap(); slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 10002, RawBlock { slot: 10002, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) .await .unwrap(); slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 10005, RawBlock { slot: 10000, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) .await .unwrap(); slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 10006, RawBlock { slot: 10000, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) @@ -517,20 +501,17 @@ async fn test_clean_forks() { .unwrap(); // Need for SLOT_CHECK_OFFSET slot_storage - .raw_blocks_cbor + .raw_blocks .put_async( 30000, RawBlock { slot: 30000, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: "".to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) @@ -825,20 +806,17 @@ async fn test_process_forked_transaction() { // // for this test all we need is key from Rocks raw_blocks_cbor column family, so RawBlock data could be arbitrary slot_storage - .raw_blocks_cbor + .raw_blocks .put( slot_normal_tx, RawBlock { slot: slot_normal_tx, - block: solana_transaction_status::UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "previousBlockHash".to_string(), blockhash: "blockHash".to_string(), parent_slot: slot_normal_tx, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) @@ -847,20 +825,17 @@ async fn test_process_forked_transaction() { // Required for SLOT_CHECK_OFFSET // 16000 is arbitrary number slot_storage - .raw_blocks_cbor + .raw_blocks .put( slot_normal_tx + 16000, RawBlock { slot: slot_normal_tx + 16000, - block: solana_transaction_status::UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "previousBlockHash".to_string(), blockhash: "blockHash".to_string(), parent_slot: slot_normal_tx + 16000, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) diff --git a/nft_ingester/tests/gapfiller_tests.rs b/nft_ingester/tests/gapfiller_tests.rs index d2afad228..122c86d8e 100644 --- a/nft_ingester/tests/gapfiller_tests.rs +++ b/nft_ingester/tests/gapfiller_tests.rs @@ -1,6 +1,6 @@ use std::sync::Arc; -use entities::models::{AssetCompleteDetailsGrpc, Updated}; +use entities::models::{AssetCompleteDetailsGrpc, RawBlockWithTransactions, Updated}; use futures::stream; use interface::asset_streaming_and_discovery::{ AsyncError, MockAssetDetailsConsumer, MockRawBlocksConsumer, @@ -12,7 +12,6 @@ use rocks_db::{ generated::asset_generated::asset as fb, migrator::MigrationState, SlotStorage, Storage, }; use solana_sdk::pubkey::Pubkey; -use solana_transaction_status::UiConfirmedBlock; use tempfile::TempDir; use tokio::{sync::Mutex, task::JoinSet}; @@ -96,15 +95,12 @@ async fn test_process_raw_blocks_stream() { let blockhash = "blockhash"; let block = entities::models::RawBlock { slot, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: blockhash.to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }; let mut mock = MockRawBlocksConsumer::new(); @@ -116,6 +112,6 @@ async fn test_process_raw_blocks_stream() { // process_raw_blocks_stream(rx, storage.clone(), 100, 200, mock).await; - let selected_data = storage.raw_blocks_cbor.get_async(slot).await.unwrap().unwrap(); + let selected_data = storage.raw_blocks.get_async(slot).await.unwrap().unwrap(); assert_eq!(selected_data.block.blockhash, blockhash.to_string()); } diff --git a/rocks-db/Cargo.toml b/rocks-db/Cargo.toml index e0f7882b3..6462e5193 100644 --- a/rocks-db/Cargo.toml +++ b/rocks-db/Cargo.toml @@ -6,6 +6,7 @@ edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html [dependencies] +clap = { workspace = true } rocksdb = { workspace = true } solana-sdk = { workspace = true } bincode = { workspace = true } @@ -75,3 +76,6 @@ name = "fork_detector" [[bin]] name = "leaf_checker" + +[[bin]] +name = "migrate_slots_db" diff --git a/rocks-db/src/bin/fork_detector/main.rs b/rocks-db/src/bin/fork_detector/main.rs index cb965237b..7e58e940c 100644 --- a/rocks-db/src/bin/fork_detector/main.rs +++ b/rocks-db/src/bin/fork_detector/main.rs @@ -151,7 +151,7 @@ async fn check_assets_signatures( let higher_seq_slot = if last_sig.1 > before_last_sig.1 { last_sig.2 } else { before_last_sig.2 }; - match slots_db.raw_blocks_cbor.has_key(higher_seq_slot).await { + match slots_db.raw_blocks.has_key(higher_seq_slot).await { Ok(has_block) => { if !has_block { // only block check is not enough because was found out that during forks diff --git a/rocks-db/src/bin/migrate_slots_db/README.md b/rocks-db/src/bin/migrate_slots_db/README.md new file mode 100644 index 000000000..693991701 --- /dev/null +++ b/rocks-db/src/bin/migrate_slots_db/README.md @@ -0,0 +1,33 @@ +# Slot storage migrator + +## Quick overview + +This exists to migrate from the old, deprecated format of slots, containing UiConfirmedBlock, to a new one, containing a smaller custom data structure that fits our needs. + +The function of the migration is pretty simple: + +1. Open the slots database in primary mode with the deprecated CF descriptor +2. Start iterating on the deprecated blocks +3. In parallel, decode the bytes into the deprecated structure, convert it to the new format, and send over a channel to the writing worker. +4. The writing worker receives the new data in (k, v) format, and writes it to a batch, which is optionally flushed to the new column family if the batch reaches the desired size. +5. Periodically free the memory of the spawned tasks once they are finished. +6. When the iterator has ended, do one final flush of the batch, and delete the old column family. + +**NOTE**: the deprecated column family is not dropped until the iterator has ended, meaning this process can be restarted from the last known slot key. + +## Known issues + +This process consumes a lot of memory due to the need of allocating additional heap memory to convert iterator items to owned values (vectors). Therefore, if the memory is not sufficient, the task might be killed by the OS. +Because of this, we must be able to start the migration task from the last known point. The process of doing this is looking at the logs, determining the last slot that was successfully inserted into rocksdb, and specifying this slot in the `start_slot` parameter. + +## Running + +To start the migration, run: +```shell +cargo r -r --bin migrate_slots_db -- --slots-db-path /path/to/slot/storage +``` + +To migrate from the specified slot, the command is largely the same, with the exception of specifying the starting slot: +```shell +cargo r -r --bin migrate_slots_db -- --slots-db-path /path/to/slot/storage --start-slot 180000000 # Replace the start slot with the desired one +``` diff --git a/rocks-db/src/bin/migrate_slots_db/main.rs b/rocks-db/src/bin/migrate_slots_db/main.rs new file mode 100644 index 000000000..c2349183c --- /dev/null +++ b/rocks-db/src/bin/migrate_slots_db/main.rs @@ -0,0 +1,143 @@ +use std::{collections::HashMap, sync::Arc}; + +use clap::Parser; +use entities::models::{RawBlock, RawBlockDeprecated}; +use metrics_utils::red::RequestErrorDurationMetrics; +use rocks_db::{column::TypedColumn, errors::StorageError, SlotStorage, Storage}; +use rocksdb::DB; +use tokio::{ + sync::{Mutex, Semaphore}, + task::{JoinError, JoinSet}, +}; + +#[derive(Parser)] +#[command(author, version, about, long_about = None)] +struct Args { + #[clap(long, help = "path to the slots database you want to migrate")] + slots_db_path: String, + #[clap(long, help = "optional starting slot for the migration")] + start_slot: Option, +} + +const PARALLEL_WORKERS: u16 = 1024; +const WRITE_BATCH_SIZE: usize = 10_000; +/// An interval which serves to await all tasks in the joinset & drain it, deallocating task +/// memory +const JOINSET_CLEARANCE_CLOCK: u32 = 1_000_000; + +fn put_batch_vec( + backend: Arc, + values: &mut Vec<(C::KeyType, C::ValueType)>, +) -> Result<(), StorageError> { + let mut batch = rocksdb::WriteBatchWithTransaction::::default(); + for (k, v) in values.drain(..) { + let serialized_value = C::encode(&v).map_err(|e| StorageError::Common(e.to_string()))?; + batch.put_cf( + &backend.cf_handle(C::NAME).unwrap(), + C::encode_key(k.clone()), + serialized_value, + ) + } + backend.write(batch)?; + Ok(()) +} + +#[tokio::main(flavor = "multi_thread")] +async fn main() { + let args = Args::parse(); + let tasks = Arc::new(Mutex::new(JoinSet::>::new())); + let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); + let cf_descriptors = Storage::cfs_to_column_families( + SlotStorage::cf_names() + .into_iter() + .chain(std::iter::once(RawBlockDeprecated::NAME)) + .collect(), + ); + let db = Arc::new( + DB::open_cf_descriptors( + &Storage::get_db_options(), + args.slots_db_path.clone(), + cf_descriptors, + ) + .expect("open rocks slot storage to migrate"), + ); + let slot_storage = SlotStorage::new(db, tasks, red_metrics); + eprintln!("Opened slots database in primary mode at {}", args.slots_db_path); + let mut iter = slot_storage.db.raw_iterator_cf( + &slot_storage.db.cf_handle(RawBlockDeprecated::NAME).expect("get raw blocks cf handle"), + ); + let batch: Arc>> = + Arc::new(Mutex::new(Vec::with_capacity(WRITE_BATCH_SIZE))); + let (tx, mut rx) = tokio::sync::mpsc::channel::<(u64, RawBlock)>(WRITE_BATCH_SIZE); + if let Some(start_slot) = args.start_slot { + iter.seek(RawBlockDeprecated::encode_key(start_slot as u64)); + } else { + iter.seek_to_first(); + } + let start = std::time::Instant::now(); + let worker_semaphore = Arc::new(Semaphore::new(PARALLEL_WORKERS as usize)); + let mut js = tokio::task::JoinSet::<()>::new(); + let handle = tokio::task::spawn({ + let batch = batch.clone(); + let slot_storage = slot_storage.clone(); + async move { + while let Some((slot, raw_block)) = rx.recv().await { + let mut batch = batch.lock().await; + batch.push((slot, raw_block)); + if batch.len() >= WRITE_BATCH_SIZE { + let ((start_slot, _), (end_slot, _)) = + (batch.first().unwrap(), batch.last().unwrap()); + eprintln!("Writing slots {} through {} to rocksdb...", start_slot, end_slot); + let db = slot_storage.db.clone(); + put_batch_vec::(db, &mut *batch) + .expect("raw blocks batch to be inserted into rocksdb"); + } + } + } + }); + + let mut keys_processed = 0u32; + + while iter.valid() { + let (k, v) = iter.item().unwrap(); + let (k, v) = (k.to_vec(), v.to_vec()); + let permit = + worker_semaphore.clone().acquire_owned().await.expect("acquire owned semaphore permit"); + let tx = tx.clone(); + js.spawn(async move { + let slot = RawBlockDeprecated::decode_key(k).expect("decode raw block key"); + let raw_block_deprecated = + RawBlockDeprecated::decode(&v).expect("decode deprecated raw block value"); + let raw_block = raw_block_deprecated.into(); + tx.send((slot, raw_block)).await.expect("rx to not be closed"); + drop(permit); + }); + keys_processed += 1; + if keys_processed >= JOINSET_CLEARANCE_CLOCK { + while js.join_next().await.is_some() {} + js.detach_all(); + keys_processed = 0; + } + iter.next(); + } + + drop(tx); + + while js.join_next().await.is_some() {} + handle.await.expect("rx to receive and process everything"); + + let mut batch = Arc::into_inner(batch).unwrap().into_inner(); + // flush what's left if the iterator has ended + if !batch.is_empty() { + let ((start_slot, _), (end_slot, _)) = (batch.first().unwrap(), batch.last().unwrap()); + eprintln!("Writing slots {} through {} to rocksdb...", start_slot, end_slot); + let map: HashMap = std::mem::take(&mut batch).into_iter().collect(); + slot_storage.raw_blocks.put_batch(map).await.expect("raw blocks batch"); + } + + slot_storage.db.drop_cf(RawBlockDeprecated::NAME).expect("delete raw blocks deprecated cf"); + slot_storage.db.flush().expect("flush data after having finished the migration"); + let elapsed = start.elapsed().as_secs_f32(); + + eprintln!("Finished migration successfully. Took {} seconds", elapsed); +} diff --git a/rocks-db/src/clients/raw_blocks_streaming_client.rs b/rocks-db/src/clients/raw_blocks_streaming_client.rs index 833985f6a..4a83bf27a 100644 --- a/rocks-db/src/clients/raw_blocks_streaming_client.rs +++ b/rocks-db/src/clients/raw_blocks_streaming_client.rs @@ -48,8 +48,7 @@ async fn process_raw_blocks_range( for pair in iterator { let (key, value) = pair.map_err(|e| Box::new(e) as AsyncError)?; - let block = serde_cbor::from_slice::(value.as_ref()) - .map_err(|e| Box::new(e) as AsyncError)?; + let block = RawBlock::decode(value.as_ref()).map_err(|e| Box::new(e) as AsyncError)?; let slot = RawBlock::decode_key(key.to_vec()).map_err(|e| Box::new(e) as AsyncError)?; if slot > end_slot { break; @@ -70,7 +69,7 @@ impl RawBlockGetter for Storage { .map_err(|e| Box::new(e) as AsyncError) .and_then(|res| { let err_msg = format!("Cannot get raw block with slot: '{slot}'!"); - res.and_then(|r| serde_cbor::from_slice::(r.as_slice()).ok()) + res.and_then(|r| RawBlock::decode(r.as_slice()).ok()) .ok_or(Box::new(StorageError::NotFound(err_msg)) as AsyncError) }) } diff --git a/rocks-db/src/column.rs b/rocks-db/src/column.rs index 680c372dd..ccf2f4ffc 100644 --- a/rocks-db/src/column.rs +++ b/rocks-db/src/column.rs @@ -34,7 +34,7 @@ pub trait TypedColumn { } } -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct Column where C: TypedColumn, diff --git a/rocks-db/src/columns/raw_block.rs b/rocks-db/src/columns/raw_block.rs index 5934b7ae0..0faaaf4c3 100644 --- a/rocks-db/src/columns/raw_block.rs +++ b/rocks-db/src/columns/raw_block.rs @@ -1,14 +1,14 @@ use std::sync::Arc; use async_trait::async_trait; -use entities::models::RawBlock; +use entities::models::{RawBlock, RawBlockDeprecated, RawBlockWithTransactions}; use interface::{ error::StorageError as InterfaceStorageError, signature_persistence::BlockProducer, }; use crate::{column::TypedColumn, errors::StorageError, key_encoders, SlotStorage}; -impl TypedColumn for RawBlock { +impl TypedColumn for RawBlockDeprecated { type KeyType = u64; type ValueType = Self; @@ -31,15 +31,30 @@ impl TypedColumn for RawBlock { } } +impl TypedColumn for RawBlock { + type KeyType = u64; + + type ValueType = Self; + const NAME: &'static str = "RAW_BLOCK"; + + fn encode_key(slot: u64) -> Vec { + key_encoders::encode_u64(slot) + } + + fn decode_key(bytes: Vec) -> crate::Result { + key_encoders::decode_u64(bytes) + } +} + #[async_trait] impl BlockProducer for SlotStorage { async fn get_block( &self, slot: u64, backup_provider: Option>, - ) -> Result { + ) -> Result { let raw_block = self - .raw_blocks_cbor + .raw_blocks .get_async(slot) .await .map_err(|e| InterfaceStorageError::Common(e.to_string()))?; diff --git a/rocks-db/src/fork_cleaner.rs b/rocks-db/src/fork_cleaner.rs index 088b1aa6b..8b2f9441c 100644 --- a/rocks-db/src/fork_cleaner.rs +++ b/rocks-db/src/fork_cleaner.rs @@ -13,7 +13,7 @@ use crate::{ column::TypedColumn, columns::{cl_items::ClItemV2, leaf_signatures::LeafSignature}, SlotStorage, Storage, DROP_ACTION, FULL_ITERATION_ACTION, ITERATOR_TOP_ACTION, - RAW_BLOCKS_CBOR_ENDPOINT, ROCKS_COMPONENT, + RAW_BLOCKS_ENDPOINT, ROCKS_COMPONENT, }; #[async_trait] @@ -126,7 +126,7 @@ impl ForkChecker for SlotStorage { self.red_metrics.observe_request( ROCKS_COMPONENT, FULL_ITERATION_ACTION, - RAW_BLOCKS_CBOR_ENDPOINT, + RAW_BLOCKS_ENDPOINT, start_time, ); @@ -141,7 +141,7 @@ impl ForkChecker for SlotStorage { self.red_metrics.observe_request( ROCKS_COMPONENT, ITERATOR_TOP_ACTION, - RAW_BLOCKS_CBOR_ENDPOINT, + RAW_BLOCKS_ENDPOINT, start_time, ); // if there are no saved blocks - we can not do any checks @@ -150,7 +150,7 @@ impl ForkChecker for SlotStorage { self.red_metrics.observe_request( ROCKS_COMPONENT, ITERATOR_TOP_ACTION, - RAW_BLOCKS_CBOR_ENDPOINT, + RAW_BLOCKS_ENDPOINT, start_time, ); it.key().and_then(|b| RawBlock::decode_key(b.to_vec()).ok()).unwrap_or_default() diff --git a/rocks-db/src/lib.rs b/rocks-db/src/lib.rs index bff8f8c88..b0205a8fd 100644 --- a/rocks-db/src/lib.rs +++ b/rocks-db/src/lib.rs @@ -66,15 +66,16 @@ pub type Result = std::result::Result; const ROCKS_COMPONENT: &str = "rocks_db"; const DROP_ACTION: &str = "drop"; -const RAW_BLOCKS_CBOR_ENDPOINT: &str = "raw_blocks_cbor"; +const RAW_BLOCKS_ENDPOINT: &str = "raw_blocks"; const FULL_ITERATION_ACTION: &str = "full_iteration"; const BATCH_ITERATION_ACTION: &str = "batch_iteration"; const BATCH_GET_ACTION: &str = "batch_get"; const ITERATOR_TOP_ACTION: &str = "iterator_top"; const MAX_WRITE_BUFFER_SIZE: u64 = 256 * 1024 * 1024; // 256MB +#[derive(Clone)] pub struct SlotStorage { pub db: Arc, - pub raw_blocks_cbor: Column, + pub raw_blocks: Column, join_set: Arc>>>, red_metrics: Arc, } @@ -85,8 +86,8 @@ impl SlotStorage { join_set: Arc>>>, red_metrics: Arc, ) -> Self { - let raw_blocks_cbor = Storage::column(db.clone(), red_metrics.clone()); - Self { db, raw_blocks_cbor, red_metrics, join_set } + let raw_blocks = Storage::column(db.clone(), red_metrics.clone()); + Self { db, raw_blocks, red_metrics, join_set } } pub fn cf_names() -> Vec<&'static str> { @@ -358,7 +359,7 @@ impl Storage { Ok(Self::new(db, join_set, red_metrics)) } - fn cfs_to_column_families(cfs: Vec<&str>) -> Vec { + pub fn cfs_to_column_families(cfs: Vec<&str>) -> Vec { cfs.iter() .map(|name| ColumnFamilyDescriptor::new(*name, Self::get_default_cf_options())) .collect() @@ -428,7 +429,7 @@ impl Storage { Column { backend, column: PhantomData, red_metrics } } - fn get_db_options() -> Options { + pub fn get_db_options() -> Options { let mut options = Options::default(); // Create missing items to support a clean start diff --git a/rocks-db/src/transaction.rs b/rocks-db/src/transaction.rs index b9a081a89..4784d06f8 100644 --- a/rocks-db/src/transaction.rs +++ b/rocks-db/src/transaction.rs @@ -1,5 +1,5 @@ use async_trait::async_trait; -use entities::models::{BatchMintToVerify, BufferedTransaction, SignatureWithSlot}; +use entities::models::{BatchMintToVerify, SignatureWithSlot, TransactionInfo}; use interface::error::StorageError; use solana_sdk::pubkey::Pubkey; use spl_account_compression::{events::ChangeLogEventV1, state::PathNode}; @@ -13,7 +13,7 @@ use crate::{ pub trait TransactionProcessor: Sync + Send + 'static { fn get_ingest_transaction_results( &self, - tx: BufferedTransaction, + tx: TransactionInfo, ) -> Result; } diff --git a/rocks-db/tests/asset_streaming_client_tests.rs b/rocks-db/tests/asset_streaming_client_tests.rs index da2295596..dc3b5e90d 100644 --- a/rocks-db/tests/asset_streaming_client_tests.rs +++ b/rocks-db/tests/asset_streaming_client_tests.rs @@ -2,11 +2,10 @@ mod tests { use std::collections::HashSet; - use entities::models::RawBlock; + use entities::models::{RawBlock, RawBlockWithTransactions}; use interface::asset_streaming_and_discovery::{AssetDetailsStreamer, RawBlocksStreamer}; use setup::rocks::*; use solana_sdk::pubkey::Pubkey; - use solana_transaction_status::UiConfirmedBlock; use tokio_stream::StreamExt; #[tokio::test] @@ -107,20 +106,17 @@ mod tests { let slot = 153; let blockhash = "blockhash"; slot_storage - .raw_blocks_cbor + .raw_blocks .put( slot, RawBlock { slot, - block: UiConfirmedBlock { + block: RawBlockWithTransactions { previous_blockhash: "".to_string(), blockhash: blockhash.to_string(), parent_slot: 0, - transactions: None, - signatures: None, - rewards: None, + transactions: Default::default(), block_time: None, - block_height: None, }, }, ) diff --git a/rocks-db/tests/raw_block_tests.rs b/rocks-db/tests/raw_block_tests.rs index 8e78b8ca5..cc6134b95 100644 --- a/rocks-db/tests/raw_block_tests.rs +++ b/rocks-db/tests/raw_block_tests.rs @@ -1,8 +1,122 @@ #[cfg(test)] mod tests { + use entities::models::{RawBlock, RawBlockDeprecated, RawBlockWithTransactions}; + use rocks_db::column::TypedColumn; + use setup::rocks::RocksTestEnvironment; + + const RAW_BLOCK_DEPRECATED_CF_NAME: &str = "RAW_BLOCK_CBOR_ENCODED"; + + #[tokio::test] + async fn test_raw_block_encoding() { + // case 1: mock data + let raw_block_deprecated = RawBlockDeprecated { + slot: 1, + block: solana_transaction_status::UiConfirmedBlock { + previous_blockhash: "prev".to_owned(), + blockhash: "hash".to_owned(), + parent_slot: 1, + transactions: None, + signatures: None, + rewards: None, + block_time: None, + block_height: None, + }, + }; + let raw_block: RawBlock = raw_block_deprecated.into(); + + assert_eq!( + raw_block, + RawBlock { + slot: 1, + block: RawBlockWithTransactions { + blockhash: "hash".to_owned(), + previous_blockhash: "prev".to_owned(), + parent_slot: 1, + block_time: None, + transactions: Default::default(), + } + } + ); + + let encoded = RawBlock::encode(&raw_block).expect("encode raw block"); + assert_eq!(raw_block, RawBlock::decode(&encoded).expect("decode raw block")); + + // case 2: real data + let raw_block_deprecated_cbor_bytes = solana_sdk::bs58::decode("2CRX7DcPTkHEntJyANdPRifCHZ2gzQeqTNU5ewnsMfrnfNQiN5E2ESCSXNmonK9er1fiBwoL3bhxDWCAgqemzLVeg4u6ood5THJbsBofXFUi9vJv7KeP79h8hr7JZAmBiuQrccH7saUFHnTaFAiBjapG1XArWbfPvAk8VUhFwjatvpHngGc9u87LLhQm58uyXqiSWcB2JhPjb8LjhaMxEx9kC4me3g7pM6ZW2boc48H2TaRaRgzrmXkwuMeTPcuwtLRwNqGYALJ5F7gA5afBbMK7Xsue9nLjRyANcJ8857SuirZLGkz21vxdkAqECEihF6XYVexiEpr8k8WbnqqF6iLBaa6V5miZo1FWvRYuMqUCjwXmLaaUCgmQcyHF8oodQXmyqCwC23oC7qWVfFNQFeRs1KXeoTE36aXSsE1ufmDbqKMWqoUw2NyURgymgE7uk4fT6ez8s34ZUzFJMXkUuEV9Q8M9p3waXZ1Bm1kmhyys2kFM2Ac7tYhxrp7FMK8ir6kc83xSAyAYF9QnP3rzwYXbJ5WCEqPL7cwq73PkxgzDj5U4ubGJo33JumaUD4YMym3vRNVqHDU7JtzVCGFrJzC2nBLCMKnPKNWzkkM1ZmbnqHao8UkuuiSR5H44ha1zrDFYA4kbtfbs1aTq3DvZNQuudkGWqjYnwTgqGr5dF4TPf5rkZN7MpcWHudp3F1Tec9qEqieH7mFupurKfvDCR6cXprUBt3aDe1RhU7QhmEYnE9LV1MnC6YhTG1zYsBuAck6r3zCNLJJR3bViMYj4u2Dj5NUqsXowaMYoT6yZhb1iPzviF478eKCnN5aBGhmEuP4GyzayiRnt2nsZShxBqwtresMEFbPyJcgJwKVxkkLK61DbvGbW8EHmap5ECuG9dVJVhs6sxbWo4T5S51TG8BHVANbr8UmJjNiMjKqtrgp3jAXDW8feCyWUXpdN8qjjGv3jjZMj2mpaAL3zP8okk2aeGtENGTuSncbvWJ5EKkmWWhveb9ecg8meuGjhVgu9Biw4N19R3Yj4pnQp1A6FpQvfFFbCCwR7LP7y8diNbVycnFVHdt3qsrAeYjMUTffq2M8Bz8rBC7URCrJ9jsHtP1b4sVWcxexPBQaV3Fc2hBSgDsnFpuWcJBeWuj6XfgjiN2h5KLnyVoATUEqpmaAoEcwPGypDjoYmSNvAk2U63iRUM49NEE5d8KKWzKaKQ9DyrphgG4NNSrD6KsJFZVGaUntgycrdRKbqcS62fhSisrHwnMkX4sPqxHpCXGCYUvZVxnSMoB61e2TvUg3Q5cXxdx2kPREYLffTPcn5TV9cC47acibWjHqQJjRiwodjE3pRCRNyVi1uL1BsAJboNAzJgVELoZNnGx54zXyGaAhTA57FzboQsL6Sr9pxD5SvxfCVrcBodtuCsH5Ck9fkpiEyqCYX5LUoHWazSHFDi1r12sbSmC63vdnk7wcLyPY4BTaoRXJmmmrSkm5RnpRVKLaGFbKdQvVtbAVUQNHKJ3Xf6m3QhVgpYTwNfBdNdvJcDpKK69gscnzFnoNkvFFTk9NDVtVRm426p5AnHzTnwA7Txp47XMoJ711jiucaMo8FjuT5kAk2cG628bDpBKrdeV1wdrSUTq3nxYN3airZyPHNaXWKYWkbTMzACTN5it9hAjrduvYPHLu5LyeArmFtBpCY5msuajqR5AQ3QQbKjRYsCHmDPSXuRYoo7n6uDbmUwiMpsU4wP2RDMJoFTL3RavMRsG4UbtVs2zGmHR4EY5zHmLfoySaYMBWG2TSMDzFmidMQAnSyHLJjp3yZjKTxARqipRjxGVXrJR81uuXDwaF8JwCoc3SPmf1hZDxQdqnVAdhaVgUMoH8XooE9Jv781ydnFHxBbxJJm4DRvjqGXx4qcuZ5rfJx7bdaJGa37wCqycMsGd24QwFYiuevLBTcC3QurzdMXLHRH8iAqVSda86ob8ye7oMpPK7s4RUikVJSC8nNmQ8sQMEaXYKqbNrLt1TbX7mhZR1EBXfnVRjZvjwR7T7GJX3CmxUm43D29VhiCXuR86wqQnFgB6kGLudZNg7jbKgvroPvMsZX5GRPd4Ww2BZcRJKMcoL9tfCJ3ByZX1GUbbUof6fNaxxa1krMpdURfzbyx8uQ3bUkzLkftdtDttnLutminZfxu3rcA9CmNfuNrxK9vqCr3J7LxM6oMT2scpww5WpA4PeyJxGWYGurW2ZVPadX4DMMbDBmKYwgdMTcG9oGsrBhcvfA6eZijcp8Sqoxt8yRpFMYfFfiErfdMZP1QdoDRJBjqvqYXqG9e5vK8vBKqTMhzxsav").into_vec().unwrap(); + let raw_block_deprecated: RawBlockDeprecated = + serde_cbor::from_slice(&raw_block_deprecated_cbor_bytes).unwrap(); + + let raw_block: RawBlock = raw_block_deprecated.into(); + let encoded = RawBlock::encode(&raw_block).unwrap(); + assert_eq!(RawBlock::decode(&encoded).unwrap(), raw_block); + } #[tokio::test] - #[tracing_test::traced_test] - #[ignore = "already processed slots are not stored anymore"] - async fn test_get_raw_block_on_empty_db() {} + async fn test_raw_block_migration() { + let env1 = RocksTestEnvironment::new(&[]); + let raw_block_deprecated = RawBlockDeprecated { + slot: 1, + block: solana_transaction_status::UiConfirmedBlock { + previous_blockhash: "prev".to_owned(), + blockhash: "hash".to_owned(), + parent_slot: 1, + transactions: None, + signatures: None, + rewards: None, + block_time: None, + block_height: None, + }, + }; + env1.slot_storage + .db + .create_cf(RAW_BLOCK_DEPRECATED_CF_NAME, &rocksdb::Options::default()) + .expect("create deprecated cf"); + env1.slot_storage + .db + .put_cf( + &env1.slot_storage.db.cf_handle(RAW_BLOCK_DEPRECATED_CF_NAME).unwrap(), + RawBlockDeprecated::encode_key(1), + RawBlockDeprecated::encode(&raw_block_deprecated).expect("encode raw block"), + ) + .expect("put raw block into the storage"); + assert_eq!( + RawBlockDeprecated::decode( + &env1 + .slot_storage + .db + .get_cf( + &env1.slot_storage.db.cf_handle(RAW_BLOCK_DEPRECATED_CF_NAME).unwrap(), + RawBlockDeprecated::encode_key(1) + ) + .unwrap() + .unwrap() + ) + .unwrap(), + raw_block_deprecated + ); + let raw_block: RawBlock = raw_block_deprecated.into(); + env1.slot_storage + .db + .put_cf( + &env1.slot_storage.db.cf_handle(RawBlock::NAME).unwrap(), + RawBlock::encode_key(1), + RawBlock::encode(&raw_block).expect("encode raw block"), + ) + .expect("put raw block into the storage"); + assert_eq!( + RawBlock::decode( + &env1 + .slot_storage + .db + .get_cf( + &env1.slot_storage.db.cf_handle(RawBlock::NAME).unwrap(), + RawBlock::encode_key(1) + ) + .unwrap() + .unwrap() + ) + .unwrap(), + raw_block + ); + } } diff --git a/usecase/Cargo.toml b/usecase/Cargo.toml index 2b7a7b830..846ec9fbd 100644 --- a/usecase/Cargo.toml +++ b/usecase/Cargo.toml @@ -31,6 +31,6 @@ plerkle_serialization = { workspace = true } async-recursion = { workspace = true } thiserror = { workspace = true } bubblegum-batch-sdk = { workspace = true } - -[dev-dependencies] +solana-storage-proto = { workspace = true } +prost = "0.11.9" futures = { workspace = true } diff --git a/usecase/src/bigtable.rs b/usecase/src/bigtable.rs index 604ce42e3..0b5f6d4e8 100644 --- a/usecase/src/bigtable.rs +++ b/usecase/src/bigtable.rs @@ -1,6 +1,8 @@ use std::{sync::Arc, time::Duration}; use async_trait::async_trait; +use entities::models::{RawBlockWithTransactions, TransactionInfo}; +use futures::{stream, StreamExt, TryStreamExt}; use interface::{ error::{StorageError, UsecaseError}, signature_persistence::BlockProducer, @@ -8,8 +10,7 @@ use interface::{ use solana_bigtable_connection::{bigtable::BigTableConnection, CredentialType}; use solana_storage_bigtable::{LedgerStorage, DEFAULT_APP_PROFILE_ID, DEFAULT_INSTANCE_NAME}; use solana_transaction_status::{ - BlockEncodingOptions, EncodedTransactionWithStatusMeta, TransactionDetails, - TransactionWithStatusMeta, + ConfirmedBlock, EncodedTransactionWithStatusMeta, TransactionWithStatusMeta, }; use tracing::{error, warn}; @@ -33,12 +34,17 @@ impl BigTableClient { big_table_creds: String, big_table_timeout: u32, ) -> Result { - let big_table_client = LedgerStorage::new( - true, - Some(Duration::from_secs(big_table_timeout as u64)), - Some(big_table_creds.to_string()), - ) - .await?; + let big_table_client = + LedgerStorage::new_with_config(solana_storage_bigtable::LedgerStorageConfig { + read_only: true, + timeout: Some(Duration::from_secs(big_table_timeout as u64)), + credential_type: solana_storage_bigtable::CredentialType::Filepath(Some( + big_table_creds.to_string(), + )), + max_message_size: 1024 * 1024 * 1024, // 1 * 1024 * 1024 * 1024 + ..solana_storage_bigtable::LedgerStorageConfig::default() + }) + .await?; let big_table_inner_client = BigTableConnection::new( DEFAULT_INSTANCE_NAME, @@ -60,7 +66,7 @@ impl BlockProducer for BigTableClient { &self, slot: u64, _backup_provider: Option>, - ) -> Result { + ) -> Result { let mut counter = GET_DATA_FROM_BG_RETRIES; loop { @@ -80,21 +86,106 @@ impl BlockProducer for BigTableClient { }; block.transactions.retain(is_bubblegum_transaction); - let encoded: solana_transaction_status::UiConfirmedBlock = block - .clone() - .encode_with_options( - solana_transaction_status::UiTransactionEncoding::Base58, - BlockEncodingOptions { - transaction_details: TransactionDetails::Full, - show_rewards: false, - max_supported_transaction_version: Some(u8::MAX), - }, - ) - .map_err(|e| StorageError::Common(e.to_string()))?; - return Ok(encoded); + return Ok(confirmed_block_to_raw(slot, block)); } } } +fn slot_to_key(slot: u64) -> solana_bigtable_connection::bigtable::RowKey { + format!("{slot:016x}") +} +// Reverse of `slot_to_key` +fn key_to_slot(key: &str) -> Option { + match solana_program::clock::Slot::from_str_radix(key, 16) { + Ok(slot) => Some(slot), + Err(err) => { + // bucket data is probably corrupt + warn!("Failed to parse object key as a slot: {}: {}", key, err); + None + }, + } +} + +pub async fn get_blocks( + connection: &BigTableConnection, + slots: &[u64], +) -> Result, StorageError> { + // 1) Fetch raw data from Bigtable (same as before) + let row_keys = slots.iter().map(|slot| slot_to_key(*slot)).collect::>(); + let mut client = connection.client(); + + let rows = client + .get_multi_row_data("blocks", &row_keys) + .await + .map_err(|e| StorageError::Common(e.to_string()))?; + + // 2) Turn the Vec of (row_key, Vec<(colName, bytes)>) into a stream + // so we can process them in parallel. + let concurrency_limit = 4; // tweak as appropriate + + let results_stream = stream::iter(rows) + .map(|(row_key, block_cell_data)| async move { + // We'll do all the CPU-heavy steps in this async block. + // If you want to run truly CPU-bound work on a dedicated thread, + // you could also wrap each step in `spawn_blocking`. + + // Convert row_key -> slot + let slot = key_to_slot(&row_key).ok_or_else(|| { + StorageError::Common(format!("Failed to parse slot from key: {row_key}")) + })?; + + // Find the "proto" column + let (_, proto_bytes) = + block_cell_data.into_iter().find(|(name, _)| name == "proto").ok_or_else(|| { + StorageError::Common(format!("Missing 'proto' column for slot={slot}")) + })?; + + // Decompress + let data = + solana_bigtable_connection::compression::decompress(&proto_bytes).map_err(|e| { + StorageError::Common(format!("Decompress failed for slot={slot}: {e}")) + })?; + + // Decode Protobuf -> ConfirmedBlock + let block_proto: solana_storage_proto::convert::generated::ConfirmedBlock = + prost::Message::decode(&data[..]).map_err(|e| { + StorageError::Common(format!("Protobuf decode failed for slot={slot}: {e}")) + })?; + + let confirmed_block: ConfirmedBlock = block_proto.try_into().map_err(|e| { + StorageError::Common(format!("try_into failed for slot={slot}: {e}")) + })?; + + // If we get here, success + Ok::<_, StorageError>((slot, confirmed_block_to_raw(slot, confirmed_block))) + }) + // 3) Run up to `concurrency_limit` tasks in parallel. + .buffer_unordered(concurrency_limit); + + // 4) Collect the stream of Results into a single Result, _> + let blocks = results_stream.try_collect().await?; // if any item is Err(..), we fail fast + + Ok(blocks) +} + +fn confirmed_block_to_raw( + slot: u64, + mut confirmed_block: ConfirmedBlock, +) -> RawBlockWithTransactions { + // Filter out non-bubblegum transactions + confirmed_block.transactions.retain(is_bubblegum_transaction); + let transactions = confirmed_block + .transactions + .into_iter() + .filter_map(|t| TransactionInfo::from_transaction_with_status_meta_and_slot(t, slot)) + .collect(); + RawBlockWithTransactions { + blockhash: confirmed_block.blockhash, + parent_slot: confirmed_block.parent_slot, + previous_blockhash: confirmed_block.previous_blockhash, + transactions, + block_time: confirmed_block.block_time.and_then(|t| t.try_into().ok()), + } +} fn is_bubblegum_transaction(tx: &TransactionWithStatusMeta) -> bool { let meta = if let Some(meta) = tx.get_status_meta() { @@ -158,3 +249,7 @@ pub fn is_bubblegum_transaction_encoded(tx: &EncodedTransactionWithStatusMeta) - } false } + +pub fn is_bubblegum_transaction_from_info(tx: &TransactionInfo) -> bool { + tx.account_keys.iter().any(|p| *p == mpl_bubblegum::programs::MPL_BUBBLEGUM_ID) +} From a2472a9df80ccf5be97c9fa82afdf110cd093379 Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Tue, 25 Feb 2025 15:56:15 +0200 Subject: [PATCH 03/21] chore(ci): clean up env some more & downsize compose (#425) * chore(ci): clean up env some more & downsize compose * Replace the identical methods with parse_json (#428) --------- Co-authored-by: Kyrylo Stepanov --- .env.example | 98 +++++---------------- docker-compose.yaml | 23 +++-- nft_ingester/src/bin/slot_persister/main.rs | 27 +++--- nft_ingester/src/config.rs | 28 ++---- 4 files changed, 54 insertions(+), 122 deletions(-) diff --git a/.env.example b/.env.example index 627714f61..4ca2f12a1 100644 --- a/.env.example +++ b/.env.example @@ -1,25 +1,40 @@ RPC_HOST='https://mainnet-aura.metaplex.com/{personal_rpc_key}' +#Configurable app parts that could be enabled or disabled. (values in the default positions) +RUN_API=true +RUN_BACKFILLER=true +RUN_BUBBLEGUM_BACKFILLER=true +RUN_GAPFILLER=false +SHOULD_REINGEST=false +RUN_PROFILING=false +RESTORE_ROCKS_DB=false +ENABLE_ROCKS_MIGRATION=true +CHECK_PROOFS=false +SKIP_CHECK_TREE_GAPS=false + #Postgres PG_DATABASE_URL='postgres://solana:solana@localhost:5432/aura_db' +POSTGRE_DB_PATH="/postgre/db/path" #Redis REDIS_CONNECTION_CONFIG='{"redis_connection_str":"redis://127.0.0.1:6379/0"}' #RocksDB ROCKS_DB_PATH="/usr/src/rocksdb-data" -ROCKS_DB_SECONDARY_PATH="path/to/rocks/secondary/db" +ROCKS_DB_SECONDARY_PATH="/path/to/rocks/secondary/db" # path to the slots data, required for the backfiller to work ROCKS_SLOTS_DB_PATH=/path/to/slots-data ROCKS_SECONDARY_SLOTS_DB_PATH=/path/to/secondary/ingester-slots -ROCKS_ARCHIVES_DIR="path/to/rocks/backup/archives" -ROCKS_BACKUP_ARCHIVES_DIR="path/to/rocks/backup/archives" +ROCKS_ARCHIVES_DIR="/path/to/rocks/backup/archives" +ROCKS_BACKUP_ARCHIVES_DIR="/path/to/rocks/backup/archives" ROCKS_BACKUP_URL="127.0.0.1:3051/snapshot" ROCKS_MIGRATION_STORAGE_PATH=/path/to/migration_storage +ROCKS_BACKUP_DIR="/path/to/rocks/backup/" +ROCKS_DUMP_PATH="/path/to/dump" #Backfiller BACKFILLER_SOURCE_MODE=bigtable -BIG_TABLE_CONFIG='{creds="/usr/src/app/creds.json", timeout=1000}' +BIG_TABLE_CONFIG='{"creds": "/usr/src/app/creds.json", "timeout": 1000}' # Metrics port. Start HTTP server to report metrics if port exist. API_METRICS_PORT=8985 @@ -33,14 +48,10 @@ API_SERVER_PORT=8990 # GRPC Server port PEER_GRPC_PORT=9099 +GAPFILLER_PEER_ADDR="0.0.0.0" -# Docker -# Required by Postgre container -POSTGRE_DB_PATH="postgre/db/path" -ROCKS_BACKUP_DIR="path/to/rocks/backup/" -FILE_STORAGE_PATH="path/to/file/storage" +FILE_STORAGE_PATH="/path/to/file/storage" FILE_STORAGE_PATH_CONTAINER="/usr/src/app/file_storage" -ROCKS_DUMP_PATH="/path/to/dump" #Profiling (optional) PROFILING_FILE_PATH_CONTAINER="/usr/src/profiling" @@ -60,70 +71,3 @@ INTEGRITY_VERIFICATION_TEST_FILE_PATH="./test_keys/test_keys.txt" INTEGRITY_VERIFICATION_TEST_FILE_PATH_CONTAINER="/test_keys/test_keys.txt" INTEGRITY_VERIFICATION_SLOTS_COLLECT_PATH="./slots_collect" INTEGRITY_VERIFICATION_SLOTS_COLLECT_PATH_CONTAINER="/slots_collect" - -#Configurable app parts that cold be enabled or disabled. (values in the default positions) -RUN_API=true -RUN_BACKFILLER=true -RUN_BUBBLEGUM_BACKFILLER=true -RUN_GAPFILLER=false -SHOULD_REINGEST=false -RUN_PROFILING=false -RESTORE_ROCKS_DB=false -ENABLE_ROCKS_MIGRATION=true -CHECK_PROOFS=false -SKIP_CHECK_TREE_GAPS=false - -#Changes (todo Remove after review/migration) - -# API_RPC_HOST INGESTER_RPC_HOST -> RPC_HOST -#INGESTER_ROCKS_DB_PATH -> ROCKS_DB_PATH -#INGESTER_ROCKS_DB_PATH_CONTAINER -> ROCKS_DB_PATH -#INGESTER_SYNCHRONIZER_DUMP_PATH -> ROCKS_DUMP_PATH -#API_ROCKS_DB_PATH_CONTAINER -> ROCKS_DB_PATH -#INGESTER_FILE_STORAGE_PATH -> FILE_STORAGE_PATH -#INGESTER_FILE_STORAGE_PATH_CONTAINER -> FILE_STORAGE_PATH_CONTAINER -# INGESTER_PROFILING_FILE_PATH -> PROFILING_FILE_PATH -# INGESTER_PROFILING_FILE_PATH_CONTAINER -> PROFILING_FILE_PATH_CONTAINER -# INGESTER_MIGRATION_STORAGE_PATH -> ROCKS_MIGRATION_STORAGE_PATH -# -# INGESTER_ROCKS_BACKUP_ARCHIVES_DIR -> ROCKS_BACKUP_ARCHIVES_DIR -# INGESTER_ROCKS_BACKUP_DIR -> ROCKS_BACKUP_DIR -# -#SYNCHRONIZER_DUMP_PATH -> ROCKS_DUMP_PATH -#INGESTER_SLOTS_DB_PATH -> ROCKS_SLOTS_DB_PATH -#INGESTER_SECONDARY_SLOTS_DB_PATH -> ROCKS_SECONDARY_SLOTS_DB_PATH - -#API_DATABASE_CONFIG -> PG_MAX_DB_CONNECTIONS and PG_DATABASE_URL - -#SYNCHRONIZER_DUMP_PATH -> ROCKS_DUMP_PATH -#SYNCHRONIZER_ROCKS_DB_SECONDARY_PATH_CONTAINER -> ROCKS_DB_SECONDARY_PATH -#SYNCHRONIZER_DUMP_SYNCHRONIZER_BATCH_SIZE -> DUMP_SYNCHRONIZER_BATCH_SIZE -#SYNCHRONIZER_DUMP_SYNC_THRESHOLD -> DUMP_SYNC_THRESHOLD - -#API_ARCHIVES_DIR -> rocks_archives_dir -#API_ROCKS_DB_PATH_CONTAINER -> ROCKS_DB_PATH -#API_ROCKS_DB_SECONDARY_PATH_CONTAINER -> ROCKS_DB_SECONDARY_PATH -#API_FILE_STORAGE_PATH_CONTAINER -> FILE_STORAGE_PATH_CONTAINER -#API_JSON_MIDDLEWARE_CONFIG -> JSON_MIDDLEWARE_CONFIG -#API_CONSISTENCE_SYNCHRONIZATION_API_THRESHOLD - > CONSISTENCE_SYNCHRONIZATION_API_THRESHOLD -#API_CONSISTENCE_BACKFILLING_SLOTS_THRESHOLD -> CONSISTENCE_BACKFILLING_SLOTS_THRESHOLD -# -#API_SKIP_CHECK_TREE_GAPS -> SKIP_CHECK_TREE_GAPS -# INGESTER_REDIS_MESSENGER_CONFIG -> REDIS_CONNECTION_CONFIG and note the differnce (less complex type) -# INGESTER_BACKFILLER_SOURCE_MODE -> BACKFILLER_SOURCE_MODE -# INGESTER_BIG_TABLE_CONFIG -> BIG_TABLE_CONFIG -# both API_PEER_GRPC_PORT and INGESTER_PEER_GRPC_PORT were replaced with a single PEER_GRPC_PORT - only one will actually work -# API_PEER_GRPC_MAX_GAP_SLOTS and INGESTER_PEER_GRPC_MAX_GAP_SLOTS -> PEER_GRPC_MAX_GAP_SLOTS (optional) -# INGESTER_ROCKS_SYNC_INTERVAL_SECONDS and API_ROCKS_SYNC_INTERVAL_SECONDS -> ROCKS_SYNC_INTERVAL_SECONDS (optional) -# INGESTER_GAPFILLER_PEER_ADDR -> GAPFILLER_PEER_ADDR (optional, only if run_gapfiller is set) -# INGESTER_ACCOUNTS_BUFFER_SIZE -> ACCOUNT_PROCESSOR_BUFFER_SIZE (has default value, may be skipped) -# INGESTER_ACCOUNTS_PARSING_WORKERS -> REDIS_ACCOUNTS_PARSING_WORKERS -# INGESTER_ROCKS_FLUSH_BEFORE_BACKUP -> ROCKS_FLUSH_BEFORE_BACKUP -# INGESTER_ROCKS_INTERVAL_IN_SECONDS -> ROCKS_INTERVAL_IN_SECONDS -# INGESTER_TRANSACTIONS_PARSING_WORKERS -> REDIS_TRANSACTIONS_PARSING_WORKERS -# INGESTER_SHOULD_REINGEST -> SHOULD_REINGEST -# INGESTER_RUN_SEQUENCE_CONSISTENT_CHECKER -> RUN_SEQUENCE_CONSISTENT_CHECKER -# INGESTER_RUN_PROFILING -> RUN_PROFILING -# INGESTER_RUN_BUBBLEGUM_BACKFILLER -> RUN_BUBBLEGUM_BACKFILLER -# Removed: -# INGESTER_BACKFILL_RPC_ADDRESS, INGESTER_BACKFILLER_MODE, INGESTER_MESSAGE_SOURCE (redis is used as an only option as of now), INGESTER_DISABLE_SYNCHRONIZER (synchronizer is no longer part of the ingester), INGESTER_CHUNK_SIZE, INGESTER_PERMITTED_TASKS, INGESTER_TCP_CONFIG, INGESTER_WORKERS_COUNT, INGESTER_WAIT_PERIOD_SEC, INGESTER_SNAPSHOT_PARSING_WORKERS, INGESTER_SNAPSHOT_PARSING_BATCH_SIZE, INGESTER_SLOT_UNTIL, INGESTER_SLOT_START_FROM, INGESTER_RUN_FORK_CLEANER, INGESTER_RUN_DUMP_SYNCHRONIZE_ON_START diff --git a/docker-compose.yaml b/docker-compose.yaml index 9634d202f..e03ecde48 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -1,9 +1,7 @@ -version: "3.9" services: ingester: container_name: ingester restart: always - entrypoint: sh -c "./ingester --rocks-slots-db-path $$ROCKS_SLOTS_DB_PATH --rocks-secondary-slots-db-path $$ROCKS_SECONDARY_SLOTS_DB_PATH" env_file: - .env network_mode: host @@ -12,7 +10,6 @@ services: - ${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:rw - ${ROCKS_BACKUP_DIR}:${ROCKS_BACKUP_DIR}:rw - ${ROCKS_BACKUP_ARCHIVES_DIR}:${ROCKS_BACKUP_ARCHIVES_DIR}:rw - - ${PROFILING_FILE_PATH}:${PROFILING_FILE_PATH_CONTAINER}:rw - ${FILE_STORAGE_PATH}:${FILE_STORAGE_PATH_CONTAINER}:rw - ${ROCKS_MIGRATION_STORAGE_PATH}:${ROCKS_MIGRATION_STORAGE_PATH}:rw - ${ROCKS_SLOTS_DB_PATH}:${ROCKS_SLOTS_DB_PATH}:ro @@ -20,12 +17,16 @@ services: - ./creds.json:/usr/src/app/creds.json - ./migrations:/usr/src/app/migrations - ./arweave_wallet.json:/usr/src/app/arweave_wallet.json - - ./heaps:/usr/src/app/heaps:rw + # Only exists for profiling - uncomment if needed + # - ./heaps:/usr/src/app/heaps:rw + # - ${PROFILING_FILE_PATH}:${PROFILING_FILE_PATH_CONTAINER}:rw depends_on: - db build: context: . dockerfile: docker/app.Dockerfile + # Replace for profiling: + # dockerfile: docker/profiling/app.Dockerfile args: BINARY: ingester stop_grace_period: 10m @@ -44,7 +45,8 @@ services: - ${ROCKS_DB_SECONDARY_PATH}/api:${ROCKS_DB_SECONDARY_PATH}:rw - ${ROCKS_ARCHIVES_DIR}:${ROCKS_ARCHIVES_DIR}:ro - ${FILE_STORAGE_PATH}:${FILE_STORAGE_PATH_CONTAINER}:rw - - ./heaps:/usr/src/app/heaps:rw + # Only exists for profiling - uncomment if needed + # - ./heaps:/usr/src/app/heaps:rw depends_on: - db build: @@ -67,8 +69,9 @@ services: - ${ROCKS_DB_PATH}:${ROCKS_DB_PATH}:rw - ${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:rw - ${ROCKS_DB_SECONDARY_PATH}/synchronizer:${ROCKS_DB_SECONDARY_PATH}:rw - - ./heaps:/usr/src/app/heaps:rw - ./migrations:/usr/src/app/migrations + # Only exists for profiling - uncomment if needed + # - ./heaps:/usr/src/app/heaps:rw depends_on: - db build: @@ -127,16 +130,10 @@ services: slot-persister: container_name: slot-persister restart: always - entrypoint: > - sh -c "if [ -f /usr/src/app/creds.json ]; then - export BIG_TABLE_CREDENTIALS=/usr/src/app/creds.json; - fi; - exec ./slot_persister" env_file: - .env environment: SLOTS_DB_PRIMARY_PATH: ${ROCKS_SLOTS_DB_PATH} - # BIG_TABLE_CREDENTIALS: /usr/src/app/creds.json # refactored this to account for the file doesn't exist case METRICS_PORT: 9090 network_mode: host volumes: @@ -164,7 +161,7 @@ services: POSTGRES_PASSWORD: solana # The PostgreSQL password (useful to connect to the database) POSTGRES_DB: solana volumes: - - ${POSTGRE_DB_PATH:-./db-data}:/var/lib/postgresql/data/:rw + - ${POSTGRE_DB_PATH}:/var/lib/postgresql/data/:rw - ${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:${ROCKS_DUMP_PATH:-/tmp/rocks_dump}:ro logging: options: diff --git a/nft_ingester/src/bin/slot_persister/main.rs b/nft_ingester/src/bin/slot_persister/main.rs index 06dc266e9..147df6ddd 100644 --- a/nft_ingester/src/bin/slot_persister/main.rs +++ b/nft_ingester/src/bin/slot_persister/main.rs @@ -8,7 +8,11 @@ use interface::{ error::StorageError, signature_persistence::BlockProducer, slot_getter::FinalizedSlotGetter, }; use metrics_utils::{utils::start_metrics, MetricState, MetricsTrait}; -use nft_ingester::{backfiller::BackfillSource, inmemory_slots_dumper::InMemorySlotsDumper}; +use nft_ingester::{ + backfiller::BackfillSource, + config::{parse_json, BigTableConfig}, + inmemory_slots_dumper::InMemorySlotsDumper, +}; use rocks_db::{column::TypedColumn, SlotStorage}; use tokio::{ sync::{broadcast, Semaphore}, @@ -50,13 +54,9 @@ struct Args { #[arg(short, long)] start_slot: Option, - /// Big table credentials file path - #[arg(short, long, env = "BIG_TABLE_CREDENTIALS")] - big_table_credentials: Option, - - /// Optional big table timeout (default: 1000) - #[arg(short = 'B', long, default_value_t = 1000)] - big_table_timeout: u32, + /// Big table config (best passed from env) + #[arg(short, long, env, value_parser = parse_json::>)] + big_table_config: Option, /// Metrics port /// Default: 9090 @@ -195,11 +195,14 @@ pub async fn main() -> Result<(), Box> { let rpc_client = Arc::new(BackfillRPC::connect(args.rpc_host.clone())); let backfill_source = { - if let Some(ref bg_creds) = args.big_table_credentials { + if let Some(ref big_table_config) = args.big_table_config { Arc::new(BackfillSource::Bigtable(Arc::new( - BigTableClient::connect_new_with(bg_creds.clone(), args.big_table_timeout) - .await - .expect("expected to connect to big table"), + BigTableClient::connect_new_with( + big_table_config.get_big_table_creds_key().expect("get big table greds"), + big_table_config.get_big_table_timeout_key().expect("get big table timeout"), + ) + .await + .expect("expected to connect to big table"), ))) } else { Arc::new(BackfillSource::Rpc(rpc_client.clone())) diff --git a/nft_ingester/src/config.rs b/nft_ingester/src/config.rs index 7dda367d0..fb51bb6ed 100644 --- a/nft_ingester/src/config.rs +++ b/nft_ingester/src/config.rs @@ -46,7 +46,7 @@ pub struct IngesterClapArgs { #[clap(long, env = "INGESTER_PG_MAX_QUERY_TIMEOUT_SECS", default_value = "120")] pub pg_max_query_statement_timeout_secs: u32, - #[clap(short('r'), long, env, help="example: {redis_connection_str=\"redis://127.0.0.1:6379/0\"}", value_parser = parse_json_to_dict)] + #[clap(short('r'), long, env, help="example: {\"redis_connection_str\": \"redis://127.0.0.1:6379/0\"}", value_parser = parse_json::)] pub redis_connection_config: Dict, #[clap(long, env, default_value = "5")] @@ -106,7 +106,7 @@ pub struct IngesterClapArgs { )] pub run_profiling: bool, - #[clap(long, env, value_parser = parse_json_to_json_middleware_config, help = "Example: {'is_enabled':true, 'max_urls_to_parse':10} ",)] + #[clap(long, env, value_parser = parse_json::, help = "Example: {'is_enabled':true, 'max_urls_to_parse':10} ",)] pub json_middleware_config: Option, // Group: Rocks DB Configuration @@ -221,7 +221,7 @@ pub struct IngesterClapArgs { pub backfill_rpc_address: Option, #[clap(long, env, default_value = "rpc", help = "#backfiller Backfill source mode.")] pub backfiller_source_mode: BackfillerSourceMode, - #[clap(long, env, value_parser = parse_json_to_big_table_config, help ="#backfiller Big table config")] + #[clap(long, env, value_parser = parse_json::>, help ="#backfiller Big table config")] pub big_table_config: Option, #[clap( @@ -510,7 +510,7 @@ pub struct ApiClapArgs { #[clap(long, env, help = "#api Storage service base url")] pub storage_service_base_url: Option, - #[clap(long, env, value_parser = parse_json_to_json_middleware_config, help = "Example: {'is_enabled':true, 'max_urls_to_parse':10} ",)] + #[clap(long, env, value_parser = parse_json::>, help = "Example: {'is_enabled':true, 'max_urls_to_parse':10} ",)] pub json_middleware_config: Option, #[clap(long, env, default_value = "100")] pub parallel_json_downloaders: i32, @@ -525,19 +525,7 @@ pub struct ApiClapArgs { pub log_level: String, } -fn parse_json_to_dict(s: &str) -> Result { - parse_json(s) -} - -fn parse_json_to_json_middleware_config(s: &str) -> Result { - parse_json(s) -} - -fn parse_json_to_big_table_config(s: &str) -> Result { - parse_json(s) -} - -fn parse_json(s: &str) -> Result { +pub fn parse_json(s: &str) -> Result { serde_json::from_str(s).map_err(|e| format!("Failed to parse JSON: {}", e)) } @@ -568,7 +556,7 @@ pub const DATABASE_URL_KEY: &str = "url"; pub const MAX_POSTGRES_CONNECTIONS: &str = "max_postgres_connections"; #[derive(Deserialize, PartialEq, Debug, Clone)] -pub struct BigTableConfig(Dict); +pub struct BigTableConfig(serde_json::Value); pub const BIG_TABLE_CREDS_KEY: &str = "creds"; pub const BIG_TABLE_TIMEOUT_KEY: &str = "timeout"; @@ -586,7 +574,7 @@ impl BigTableConfig { } pub fn get_big_table_timeout_key(&self) -> Result { - Ok(self.0.get(BIG_TABLE_TIMEOUT_KEY).and_then(|v| v.to_u128()).ok_or( + Ok(self.0.get(BIG_TABLE_TIMEOUT_KEY).and_then(|v| v.as_u64()).ok_or( IngesterError::ConfigurationError { msg: "BIG_TABLE_TIMEOUT_KEY missing".to_string() }, )? as u32) } @@ -612,7 +600,7 @@ mod tests { "--rpc-host", "https://mainnet-aura.metaplex.com", "--redis-connection-config", - "{}", + r#"{"redis_connection_str": "foo"}"#, ]); assert_eq!(args.rocks_db_path, "./my_rocksdb"); From e8bfb5f7fa0f9a2b8caa7e4a6f3c24cf486eb223 Mon Sep 17 00:00:00 2001 From: andrii Date: Wed, 26 Feb 2025 13:32:10 +0100 Subject: [PATCH 04/21] fix update fungible parameter count --- postgre-client/src/asset_index_client.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/postgre-client/src/asset_index_client.rs b/postgre-client/src/asset_index_client.rs index 8478b8e18..19c4df855 100644 --- a/postgre-client/src/asset_index_client.rs +++ b/postgre-client/src/asset_index_client.rs @@ -28,7 +28,7 @@ pub const INSERT_ASSET_PARAMETERS_COUNT: usize = 19; pub const DELETE_ASSET_CREATOR_PARAMETERS_COUNT: usize = 2; pub const INSERT_ASSET_CREATOR_PARAMETERS_COUNT: usize = 4; pub const INSERT_AUTHORITY_PARAMETERS_COUNT: usize = 3; -pub const INSERT_FUNGIBLE_TOKEN_PARAMETERS_COUNT: usize = 4; +pub const INSERT_FUNGIBLE_TOKEN_PARAMETERS_COUNT: usize = 5; impl PgClient { pub(crate) async fn fetch_last_synced_id_impl( From 946d4e4bb295b65c9c6d2fe46b086c604b041fc6 Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Wed, 26 Feb 2025 17:28:12 +0200 Subject: [PATCH 05/21] fix(config): parse big table config into the correct type (#430) --- nft_ingester/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nft_ingester/src/config.rs b/nft_ingester/src/config.rs index fb51bb6ed..231ca63e4 100644 --- a/nft_ingester/src/config.rs +++ b/nft_ingester/src/config.rs @@ -221,7 +221,7 @@ pub struct IngesterClapArgs { pub backfill_rpc_address: Option, #[clap(long, env, default_value = "rpc", help = "#backfiller Backfill source mode.")] pub backfiller_source_mode: BackfillerSourceMode, - #[clap(long, env, value_parser = parse_json::>, help ="#backfiller Big table config")] + #[clap(long, env, value_parser = parse_json::, help ="#backfiller Big table config")] pub big_table_config: Option, #[clap( From b0cbe4005bfcc56dee4b3d02c4f22bf307da0523 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Thu, 27 Feb 2025 14:47:45 +0000 Subject: [PATCH 06/21] chore: [MTG-1360] release process (#431) * chore: set an appropriate 0.1 version * chore(ci): standardize release workflows and branch naming - Implement release preparation/finalization - Standardize branch naming to use release/v* format consistently - Update documentation to reflect automated version bumping * chore: cliff.toml nit and makefile cleanup * chore(ci): improve GitHub workflows for release process - Update docker.yml to ensure proper tag pushing with PUSH_CONDITION - Remove redundant version tag format (keeping only v-prefixed tags) - Set GH_TOKEN at job level in both release workflows - Remove unnecessary GitHub CLI authentication steps * chore(ci): use proper checkout action --- .github/workflows/docker.yml | 35 +++++- .github/workflows/release-finalize.yml | 124 +++++++++++++++++++++ .github/workflows/release-prepare.yml | 147 +++++++++++++++++++++++++ Cargo.lock | 2 +- Makefile | 8 ++ RELEASE_PROCESS.md | 75 +++++++++++++ cliff.toml | 79 +++++++++++++ nft_ingester/Cargo.toml | 2 +- 8 files changed, 466 insertions(+), 6 deletions(-) create mode 100644 .github/workflows/release-finalize.yml create mode 100644 .github/workflows/release-prepare.yml create mode 100644 RELEASE_PROCESS.md create mode 100644 cliff.toml diff --git a/.github/workflows/docker.yml b/.github/workflows/docker.yml index 9ba1eb48a..a2f31490a 100644 --- a/.github/workflows/docker.yml +++ b/.github/workflows/docker.yml @@ -8,13 +8,18 @@ on: branches: [develop] tags: ["v*"] +# Add concurrency to cancel in-progress runs on the same ref +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + # Add permissions block for GitHub Container Registry access permissions: contents: read packages: write env: - PUSH_CONDITION: ${{ github.event_name == 'push' || (github.event_name == 'workflow_dispatch' && contains(fromJSON('["refs/head/main", "refs/head/develop"]'), github.event.workflow_dispatch.ref)) }} + PUSH_CONDITION: ${{ github.event_name == 'push' && (contains(fromJSON('["refs/heads/develop"]'), github.ref) || startsWith(github.ref, 'refs/tags/')) || github.event_name == 'workflow_dispatch' && contains(fromJSON('["refs/heads/develop"]'), github.ref) }} jobs: build-base-image: @@ -23,7 +28,10 @@ jobs: version: ${{ steps.version.outputs.version }} steps: - - uses: actions/checkout@v4 + - name: Check out repository + uses: actions/checkout@v4 + with: + fetch-depth: 0 - name: Login to GitHub Container Registry uses: docker/login-action@v3 with: @@ -82,7 +90,10 @@ jobs: rocksdb_backup, ] steps: - - uses: actions/checkout@v4 # Need to checkout code for Dockerfile + - name: Check out repository + uses: actions/checkout@v4 + with: + fetch-depth: 0 - name: Login to GitHub Container Registry uses: docker/login-action@v3 with: @@ -142,7 +153,8 @@ jobs: runs-on: ubuntu-latest needs: [build-base-image, build-binary-images] steps: - - name: Repository dispatch + - name: Repository dispatch for development + if: startsWith(github.ref, 'refs/heads/develop') run: | curl -X POST \ -H "Authorization: token ${{ secrets.DISPATCH_TOKEN_DEV }}" \ @@ -155,3 +167,18 @@ jobs: "version": "${{ needs.build-base-image.outputs.version }}" } }' + + - name: Repository dispatch for production + if: startsWith(github.ref, 'refs/tags/') + run: | + curl -X POST \ + -H "Authorization: token ${{ secrets.DISPATCH_TOKEN_PROD }}" \ + -H "Accept: application/vnd.github+json" \ + https://api.github.com/repos/adm-metaex/aura-config-prod/dispatches \ + -d '{ + "event_type": "deploy", + "client_payload": { + "services": "${{ env.PUSH_CONDITION && 'ingester,slot_persister,backfill,api,synchronizer,rocksdb_backup' || '' }}", + "version": "${{ needs.build-base-image.outputs.version }}" + } + }' diff --git a/.github/workflows/release-finalize.yml b/.github/workflows/release-finalize.yml new file mode 100644 index 000000000..300e62a10 --- /dev/null +++ b/.github/workflows/release-finalize.yml @@ -0,0 +1,124 @@ +name: Finalize Release + +on: + pull_request: + types: [closed] + branches: + - main + +# Add permissions for GitHub operations +permissions: + contents: write + pull-requests: write + +jobs: + finalize-release: + if: github.event.pull_request.merged == true && startsWith(github.event.pull_request.head.ref, 'release/v') + runs-on: ubuntu-latest + env: + GH_TOKEN: ${{ secrets.GITHUB_TOKEN }} + steps: + - name: Check out repository + uses: actions/checkout@v4 + with: + fetch-depth: 0 # Fetch all history for proper tagging + + - name: Setup Git Identity + run: | + set -e + git config --global user.name "GitHub Actions" + git config --global user.email "actions@github.com" + + - name: Download changelog artifact + uses: actions/download-artifact@v4 + with: + name: changelog + path: . + + - name: Get release version + id: get_version + run: | + set -e + BRANCH_NAME="${{ github.event.pull_request.head.ref }}" + # We only support release/v* format now + VERSION=${BRANCH_NAME#release/} + echo "version=$VERSION" >> $GITHUB_OUTPUT + echo "tag_name=$VERSION" >> $GITHUB_OUTPUT + + - name: Create and push tag + run: | + set -e + git tag -a ${{ steps.get_version.outputs.tag_name }} -m "Release ${{ steps.get_version.outputs.tag_name }}" + git push origin ${{ steps.get_version.outputs.tag_name }} + # This tag push will automatically trigger the docker.yml workflow for building images + + - name: Create GitHub Release + uses: softprops/action-gh-release@v2 + with: + tag_name: ${{ steps.get_version.outputs.tag_name }} + name: Release ${{ steps.get_version.outputs.tag_name }} + body_path: CHANGELOG.md # Use the downloaded changelog + generate_release_notes: false # We're using our own changelog + draft: false + prerelease: false + + - name: Create PR to develop + run: | + set -e + gh pr create --base develop --head ${{ github.event.pull_request.head.ref }} \ + --title "Merge ${{ github.event.pull_request.head.ref }} into develop" \ + --body "Merge release branch into develop." + + - name: Merge into develop + run: | + set -e + PR_NUMBER=$(gh pr list --head ${{ github.event.pull_request.head.ref }} --base develop --json number --jq '.[0].number') + if [ -n "$PR_NUMBER" ]; then + gh pr merge --repo ${{ github.repository }} --merge --auto $PR_NUMBER + else + echo "No PR found to merge into develop" + exit 1 + fi + + # --- Post-Release Version Bump --- + + - name: Checkout develop branch for version bump + uses: actions/checkout@v4 + with: + fetch-depth: 0 # Fetch all history for proper tagging + ref: develop + + - name: Install dependencies for version bump + run: | + set -e + sudo apt-get update && sudo apt-get install -y protobuf-compiler + cargo install cargo-edit + + - name: Calculate and apply next development version + id: calculate_next_version + run: | + set -e + # Extract version without 'v' prefix + RAW_VERSION=${{ steps.get_version.outputs.version }} + VERSION_WITHOUT_V="${RAW_VERSION#v}" + + IFS='.' read -ra VERSION_PARTS <<< "$VERSION_WITHOUT_V" + MAJOR="${VERSION_PARTS[0]}" + MINOR="${VERSION_PARTS[1]}" + PATCH=$((VERSION_PARTS[2] + 1)) # Increment the patch version + NEXT_VERSION="${MAJOR}.${MINOR}.${PATCH}-dev" + echo "next_version=$NEXT_VERSION" >> $GITHUB_OUTPUT + echo "Setting develop version to $NEXT_VERSION" + + # Update Cargo.toml versions + find . -name "Cargo.toml" -type f -exec cargo set-version $NEXT_VERSION --manifest-path {} \; + + # Update any other version references + if [ -f "VERSION" ]; then + echo "$NEXT_VERSION" > VERSION + fi + + # Commit and push the version bump directly to develop + git add -A + git commit -m "chore: bump version to $NEXT_VERSION [skip ci]" + git push origin develop diff --git a/.github/workflows/release-prepare.yml b/.github/workflows/release-prepare.yml new file mode 100644 index 000000000..d0942acef --- /dev/null +++ b/.github/workflows/release-prepare.yml @@ -0,0 +1,147 @@ +name: Prepare Release + +on: + workflow_dispatch: + inputs: + version: + description: 'Version number (without v prefix, e.g. 0.5.0)' + required: true + type: string + base_commit: + description: 'Base commit SHA (leave empty to use latest develop)' + required: false + type: string + default: '' + +# Add permissions for GitHub operations +permissions: + contents: write + pull-requests: write + +jobs: + prepare-release: + runs-on: ubuntu-latest + outputs: + version: ${{ inputs.version }} # Output the version for use in other jobs + tag_name: v${{ inputs.version }} + release_branch: release/v${{ inputs.version }} + env: + GH_TOKEN: ${{ secrets.GITHUB_TOKEN }} + + steps: + - name: Check out repository + uses: actions/checkout@v4 + with: + ref: ${{ inputs.base_commit || 'develop' }} + fetch-depth: 0 # Fetch all history for changelog generation + + - name: Validate input version + run: | + set -e + # Make sure the version follows semantic versioning format + if ! echo "${{ inputs.version }}" | grep -E '^[0-9]+\.[0-9]+\.[0-9]+$'; then + echo "Error: Version must follow semantic versioning format (e.g., 0.5.0)" + exit 1 + fi + + - name: Setup Git Identity + run: | + set -e + git config --global user.name "GitHub Actions" + git config --global user.email "actions@github.com" + + - name: Install dependencies + run: | + set -e + sudo apt-get update && sudo apt-get install -y protobuf-compiler + cargo install cargo-edit + cargo install git-cliff + + - name: Create release branch + run: | + set -e + # Using standardized format: release/v{version} + RELEASE_BRANCH="release/v${{ inputs.version }}" + echo "Creating branch $RELEASE_BRANCH" + git checkout -b $RELEASE_BRANCH + echo "RELEASE_BRANCH=$RELEASE_BRANCH" >> $GITHUB_ENV + + - name: Update version numbers + run: | + set -e + # Update Cargo.toml versions + find . -name "Cargo.toml" -type f -exec cargo set-version ${{ inputs.version }} --manifest-path {} \; + + # Update any other version references (add any other files that contain version numbers) + if [ -f "VERSION" ]; then + echo "${{ inputs.version }}" > VERSION + fi + + git add -A + git commit -m "chore: bump version to ${{ inputs.version }}" + + - name: Generate changelog + id: changelog + run: | + set -e + # Generate changelog using git-cliff + git-cliff --config cliff.toml --tag "v${{ inputs.version }}" --output CHANGELOG.md + + # Generate a shorter version for PR description + git-cliff --config cliff.toml --tag "v${{ inputs.version }}" --strip header,footer > .changelog_content + + git add CHANGELOG.md + git commit -m "docs: add changelog for v${{ inputs.version }}" + + - name: Verify changelog + run: | + set -e + # Check that the changelog file exists and has content + if [ ! -s CHANGELOG.md ]; then + echo "Error: CHANGELOG.md is empty or does not exist" + exit 1 + fi + + # Check that the changelog contains the version we're releasing + if ! grep -q "v${{ inputs.version }}" CHANGELOG.md; then + echo "Error: CHANGELOG.md does not contain version v${{ inputs.version }}" + echo "Contents of CHANGELOG.md:" + cat CHANGELOG.md + exit 1 + fi + + # Check that the changelog has sections + if ! grep -q "###" CHANGELOG.md; then + echo "Warning: CHANGELOG.md does not contain any sections (###)" + echo "This might be ok if there are no conventional commits, but please verify" + fi + + echo "Changelog verification passed!" + + - name: Push release branch + run: | + set -e + git push -u origin $RELEASE_BRANCH + + - name: Create Pull Request + id: create-pr + uses: peter-evans/create-pull-request@v5 + with: + token: ${{ secrets.GITHUB_TOKEN }} + base: main + head: ${{ env.RELEASE_BRANCH }} + title: "Release v${{ inputs.version }}" + body-path: .changelog_content + draft: false + + - name: PR info + run: | + set -e + echo "Pull Request created: ${{ steps.create-pr.outputs.pull-request-url }}" + echo "Please review the PR, make any necessary adjustments, and merge when ready." + + - name: Upload changelog artifact + uses: actions/upload-artifact@v4 + with: + name: changelog + path: CHANGELOG.md diff --git a/Cargo.lock b/Cargo.lock index 3284c5272..dec6008ed 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4781,7 +4781,7 @@ dependencies = [ [[package]] name = "nft_ingester" -version = "0.7.2" +version = "0.1.0" dependencies = [ "anchor-lang 0.30.1", "arweave-rs", diff --git a/Makefile b/Makefile index 307ac35e7..04ed81ce9 100644 --- a/Makefile +++ b/Makefile @@ -47,3 +47,11 @@ clippy: test: @cargo clean -p postgre-client -p rocks-db -p interface @cargo test --features integration_tests + +# Ensure git-cliff is installed +ensure-git-cliff: + @which git-cliff > /dev/null || cargo install git-cliff + +# Generate a changelog using git-cliff +changelog: + @git-cliff --output CHANGELOG.md diff --git a/RELEASE_PROCESS.md b/RELEASE_PROCESS.md new file mode 100644 index 000000000..66ca12ef1 --- /dev/null +++ b/RELEASE_PROCESS.md @@ -0,0 +1,75 @@ +# Release Process + +This document outlines the process to follow when releasing a new version of the project. + +## Releasing a New Version + +### 1. Prepare for Release + +1. Decide on a version number for the new release following [Semantic Versioning](https://semver.org/) (e.g., `0.5.0`). + +2. Locally or using the GitHub UI, trigger the `release-prepare` workflow: + ```bash + make release VERSION=0.5.0 + ``` + or manually trigger the workflow via GitHub Actions UI. + +3. This will: + - Create a new branch `release/v0.5.0` from the latest `develop` branch + - Update version numbers in all `Cargo.toml` files to `0.5.0` + - Generate a CHANGELOG.md file + - Create a pull request from `release/v0.5.0` to `main` + +### 2. Review and Merge the Release PR + +1. Review the generated PR to ensure the changelog and version changes are correct. +2. Request reviews from other team members as necessary. +3. Make any final adjustments directly to the `release/v0.5.0` branch. +4. Once approved, merge the PR into `main`. + +### 3. Automatic Release Finalization + +When the release PR is merged to `main`, the `release-finalize` workflow will automatically: + +1. Create and push a tag for the release (e.g., `v0.5.0`) +2. Create a GitHub Release with the changelog content +3. Build and publish Docker images for the release +4. Create a PR to merge changes back to `develop` +5. Automatically merge this PR into `develop` +6. Automatically bump the version on `develop` to the next development version (e.g., `0.5.1-dev`) + +No manual intervention is required for these steps unless there are conflicts when merging back to `develop`. + +## Hotfix Process + +For urgent fixes that need to bypass the normal release flow: + +1. Create a branch `hotfix/v0.5.1` from `main` +2. Make your changes and commit them +3. Update version numbers in all `Cargo.toml` files to `0.5.1` +4. Create a PR from `hotfix/v0.5.1` to `main` +5. After the PR is merged, the same automatic finalization steps listed above will occur + +## Version Numbering + +We follow [Semantic Versioning](https://semver.org/): + +- MAJOR version for incompatible API changes +- MINOR version for backwards-compatible functionality +- PATCH version for backwards-compatible bug fixes + +Development versions on the `develop` branch have a `-dev` suffix appended to the patch number (e.g., `0.5.1-dev`). + +## Changelog Generation + +The changelog is generated automatically using [git-cliff](https://github.com/orhun/git-cliff), which parses [Conventional Commits](https://www.conventionalcommits.org/) to generate a structured changelog. + +To ensure your commits appear correctly in the changelog, prefix them with: + +- `feat:` for new features +- `fix:` for bug fixes +- `chore:` for maintenance tasks +- `docs:` for documentation updates +- `refactor:` for code refactoring +- `test:` for adding or updating tests +- `perf:` for performance improvements \ No newline at end of file diff --git a/cliff.toml b/cliff.toml new file mode 100644 index 000000000..c279ac1c4 --- /dev/null +++ b/cliff.toml @@ -0,0 +1,79 @@ +# Configuration for git-cliff changelog generator +# See https://git-cliff.org/docs/configuration + +[changelog] +# changelog header +header = """ +# Changelog\n +All notable changes to this project will be documented in this file.\n +""" +# template for the changelog body +# https://tera.netlify.app/docs +body = """ +{% if version %}\ +## [{{ version | trim_start_matches(pat="v") }}] - {{ timestamp | date(format="%Y-%m-%d") }} +{% else %}\ +## [unreleased] +{% endif %}\ + +{% if previous %}\ +{% if previous.version %}\ +[{{ version | trim_start_matches(pat="v") }}]: https://github.com/metaplex-foundation/aura/compare/{{ previous.version }}...{{ version }} +{% else %}\ +[{{ version | trim_start_matches(pat="v") }}]: https://github.com/metaplex-foundation/aura/releases/tag/{{ version }} +{% endif %}\ +{% endif %}\ + +{% for group, commits in commits | group_by(attribute="group") %} +### {{ group | upper_first }} +{% for commit in commits %} +- {% if commit.breaking %}**BREAKING**: {% endif %}{{ commit.message | upper_first }} ([{{ commit.id | truncate(length=7, end="") }}](https://github.com/metaplex-foundation/aura/commit/{{ commit.id }})){% if commit.footers %} {% for footer in commit.footers %}{{ footer }}{% endfor %}{% endif %} +{% endfor %} +{% endfor %}\n +""" +# remove the leading and trailing whitespace from the templates +trim = true +# changelog footer +footer = "" + +# postprocessors +postprocessors = [ + { pattern = '', replace = "<" }, # needed to escape jinja2 templates +] + +[git] +# parse the commits based on https://www.conventionalcommits.org +conventional_commits = true +# filter out the commits that are not conventional +filter_unconventional = true +# process each line of a commit as an individual commit +split_commits = false +# regex for parsing and grouping commits +commit_parsers = [ + { message = "^feat", group = "Features" }, + { message = "^fix", group = "Bug Fixes" }, + { message = "^doc", group = "Documentation" }, + { message = "^perf", group = "Performance" }, + { message = "^refactor", group = "Refactor" }, + { message = "^style", group = "Styling" }, + { message = "^test", group = "Testing" }, + { message = "^chore\\(release\\): prepare for", skip = true }, + { message = "^chore\\(deps\\)", group = "Dependencies" }, + { message = "^chore", group = "Miscellaneous Tasks" }, + { message = "^ci", group = "CI/CD" }, + { body = ".*security", group = "Security" }, +] +# filter out the commits that are not matched by commit parsers +filter_commits = false +# glob pattern for matching git tags +tag_pattern = "v[0-9]*.[0-9]*.[0-9]*" +# regex for skipping tags +skip_tags = "" +# regex for ignoring tags +ignore_tags = "" +# sort the tags chronologically +date_order = false +# sort the commits inside sections by oldest/newest order +sort_commits = "oldest" +# limit the number of commits included in the changelog +limit_commits = 0 diff --git a/nft_ingester/Cargo.toml b/nft_ingester/Cargo.toml index 73205f5b9..58b1858f2 100644 --- a/nft_ingester/Cargo.toml +++ b/nft_ingester/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "nft_ingester" -version = "0.7.2" +version = "0.1.0" edition = "2021" publish = false From f49ee305c77f7df3a9e25fe317f44505040bda06 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Thu, 27 Feb 2025 17:10:48 +0000 Subject: [PATCH 07/21] fix(ci): resolve issues with release workflows (#432) - Fix changelog generation in release-prepare.yml: - Remove invalid --strip header,footer parameter and use --strip all instead - Remove --config flag when cliff.toml might not exist - Add warning message when cliff.toml is missing --- .github/workflows/release-prepare.yml | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/.github/workflows/release-prepare.yml b/.github/workflows/release-prepare.yml index d0942acef..4609e47aa 100644 --- a/.github/workflows/release-prepare.yml +++ b/.github/workflows/release-prepare.yml @@ -84,11 +84,16 @@ jobs: id: changelog run: | set -e + # Check if cliff.toml exists + if [ ! -f "cliff.toml" ]; then + echo "Warning: cliff.toml not found, will use default configuration" + fi + # Generate changelog using git-cliff - git-cliff --config cliff.toml --tag "v${{ inputs.version }}" --output CHANGELOG.md + git-cliff --tag "v${{ inputs.version }}" --output CHANGELOG.md # Generate a shorter version for PR description - git-cliff --config cliff.toml --tag "v${{ inputs.version }}" --strip header,footer > .changelog_content + git-cliff --tag "v${{ inputs.version }}" --strip all > .changelog_content git add CHANGELOG.md git commit -m "docs: add changelog for v${{ inputs.version }}" From 411ea91fb639c575b3d3180e751ba46c46584f42 Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Fri, 28 Feb 2025 11:38:09 +0200 Subject: [PATCH 08/21] fix(slot_persister): change `parse_json` expected type in config (#435) --- nft_ingester/src/bin/slot_persister/main.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nft_ingester/src/bin/slot_persister/main.rs b/nft_ingester/src/bin/slot_persister/main.rs index 147df6ddd..c75a9b065 100644 --- a/nft_ingester/src/bin/slot_persister/main.rs +++ b/nft_ingester/src/bin/slot_persister/main.rs @@ -55,7 +55,7 @@ struct Args { start_slot: Option, /// Big table config (best passed from env) - #[arg(short, long, env, value_parser = parse_json::>)] + #[arg(short, long, env, value_parser = parse_json::)] big_table_config: Option, /// Metrics port From 558c7604e3c9915a0cfc6de2b78a09d47cc111d9 Mon Sep 17 00:00:00 2001 From: Andrii Date: Fri, 28 Feb 2025 16:04:55 +0100 Subject: [PATCH 09/21] [MTG-1351] fix(api): optimise get collection assets query (#434) * MTG-1351 optimise get collection assets query --- integration_tests/src/regular_nft_tests.rs | 22 +++++-------------- ..._fungible_token_mint_freeze_authority.snap | 12 ++++------ integration_tests/src/token_tests.rs | 6 ++--- .../13_update_assets_collection_index.sql | 3 +++ postgre-client/src/lib.rs | 2 +- postgre-client/src/load_client.rs | 4 +++- 6 files changed, 19 insertions(+), 30 deletions(-) create mode 100644 migrations/13_update_assets_collection_index.sql diff --git a/integration_tests/src/regular_nft_tests.rs b/integration_tests/src/regular_nft_tests.rs index bfb6d7225..31d2f5a38 100644 --- a/integration_tests/src/regular_nft_tests.rs +++ b/integration_tests/src/regular_nft_tests.rs @@ -234,18 +234,12 @@ async fn test_regular_nft_collection() { #[named] async fn test_search_by_owner_with_show_zero_balance() { let name = trim_test_name(function_name!()); - let mut wellknown_fungible_accounts = HashMap::new(); - wellknown_fungible_accounts - .insert(String::from("HxhWkVpk5NS4Ltg5nij2G671CKXFRKPK8vy271Ub4uEK"), String::from("Hxro")); - wellknown_fungible_accounts - .insert(String::from("METAewgxyPbgwsseH8T16a39CQ5VyVxZi9zXiDPY18m"), String::from("MPLX")); - let setup = TestSetup::new_with_options( name.clone(), TestSetupOptions { network: Some(Network::Mainnet), clear_db: true, - well_known_fungible_accounts: wellknown_fungible_accounts, + well_known_fungible_accounts: well_known_fungible_tokens(), }, ) .await; @@ -368,18 +362,12 @@ async fn test_search_by_owner_with_show_zero_balance() { #[named] async fn test_search_by_owner_with_show_zero_balance_false() { let name = trim_test_name(function_name!()); - let mut wellknown_fungible_accounts = HashMap::new(); - wellknown_fungible_accounts - .insert(String::from("HxhWkVpk5NS4Ltg5nij2G671CKXFRKPK8vy271Ub4uEK"), String::from("Hxro")); - wellknown_fungible_accounts - .insert(String::from("METAewgxyPbgwsseH8T16a39CQ5VyVxZi9zXiDPY18m"), String::from("MPLX")); - let setup = TestSetup::new_with_options( name.clone(), TestSetupOptions { network: Some(Network::Mainnet), clear_db: true, - well_known_fungible_accounts: wellknown_fungible_accounts, + well_known_fungible_accounts: well_known_fungible_tokens(), }, ) .await; @@ -1028,15 +1016,15 @@ async fn test_recognise_popular_fungible_tokens() { let token_price_fetcher = RaydiumTokenPriceFetcher::new(RAYDIUM_API_HOST.to_string(), CACHE_TTL, None); token_price_fetcher.warmup().await.unwrap(); - let wellknown_fungible_accounts = token_price_fetcher.get_all_token_symbols().await.unwrap(); - assert!(wellknown_fungible_accounts.len() > 0); + let well_known_fungible_accounts = token_price_fetcher.get_all_token_symbols().await.unwrap(); + assert!(well_known_fungible_accounts.len() > 0); let setup = TestSetup::new_with_options( name.clone(), TestSetupOptions { network: Some(Network::Mainnet), clear_db: true, - well_known_fungible_accounts: wellknown_fungible_accounts, + well_known_fungible_accounts, }, ) .await; diff --git a/integration_tests/src/snapshots/integration_tests__token_tests__fungible_token_mint_freeze_authority.snap b/integration_tests/src/snapshots/integration_tests__token_tests__fungible_token_mint_freeze_authority.snap index 86e016764..416242bd2 100644 --- a/integration_tests/src/snapshots/integration_tests__token_tests__fungible_token_mint_freeze_authority.snap +++ b/integration_tests/src/snapshots/integration_tests__token_tests__fungible_token_mint_freeze_authority.snap @@ -1,11 +1,11 @@ --- source: integration_tests/src/token_tests.rs -assertion_line: 38 -expression: response +assertion_line: 61 +expression: response_value.clone() snapshot_kind: text --- { - "interface": "Custom", + "interface": "FungibleToken", "id": "EPjFWdd5AufqSSqeM2qN1xzybapC8G4wEGGkZwyTDt1v", "content": { "$schema": "https://schema.metaplex.com/nft1.0.json", @@ -52,11 +52,7 @@ snapshot_kind: text "ownership_model": "token", "owner": "" }, - "supply": { - "print_max_supply": 0, - "print_current_supply": 0, - "edition_nonce": 252 - }, + "supply": null, "mutable": true, "burnt": false, "lamports": 5616720, diff --git a/integration_tests/src/token_tests.rs b/integration_tests/src/token_tests.rs index 989c5071f..f124cf3c7 100644 --- a/integration_tests/src/token_tests.rs +++ b/integration_tests/src/token_tests.rs @@ -14,8 +14,8 @@ use tokio::{ }; use crate::common::{ - index_seed_events, seed_token_mints, trim_test_name, Network, SeedEvent, TestSetup, - TestSetupOptions, + index_seed_events, seed_token_mints, trim_test_name, well_known_fungible_tokens, Network, + SeedEvent, TestSetup, TestSetupOptions, }; #[tokio::test] @@ -28,7 +28,7 @@ async fn test_fungible_token_mint_freeze_authority() { TestSetupOptions { network: Some(Network::Mainnet), clear_db: true, - well_known_fungible_accounts: HashMap::new(), + well_known_fungible_accounts: well_known_fungible_tokens(), }, ) .await; diff --git a/migrations/13_update_assets_collection_index.sql b/migrations/13_update_assets_collection_index.sql new file mode 100644 index 000000000..465bed2e7 --- /dev/null +++ b/migrations/13_update_assets_collection_index.sql @@ -0,0 +1,3 @@ +DROP INDEX IF EXISTS assets_v3_collection_is_collection_verified; + +CREATE INDEX assets_v3_collection_is_collection_verified_supply ON assets_v3(ast_collection, ast_is_collection_verified, ast_supply) WHERE ast_collection IS NOT NULL AND ast_supply IS NOT NULL; diff --git a/postgre-client/src/lib.rs b/postgre-client/src/lib.rs index 9a2a01b87..bd308d2ae 100644 --- a/postgre-client/src/lib.rs +++ b/postgre-client/src/lib.rs @@ -107,7 +107,7 @@ impl PgClient { pub async fn get_collection_size(&self, collection_key: &[u8]) -> Result { let start_time = chrono::Utc::now(); - let resp = sqlx::query("SELECT COUNT(*) FROM assets_v3 WHERE ast_collection = $1 AND ast_is_collection_verified = true") + let resp = sqlx::query("SELECT COUNT(*) FROM assets_v3 WHERE ast_collection = $1 AND ast_is_collection_verified = true AND ast_supply > 0") .bind(collection_key) .fetch_one(&self.pool).await; diff --git a/postgre-client/src/load_client.rs b/postgre-client/src/load_client.rs index f753bedaf..c70b470c8 100644 --- a/postgre-client/src/load_client.rs +++ b/postgre-client/src/load_client.rs @@ -165,7 +165,9 @@ impl PgClient { "asset_creators_v3_creator", "assets_authority", "assets_v3_authority_fk", + // For backward compatibility with the current state of the database, it can be removed over time "assets_v3_collection_is_collection_verified", + "assets_v3_collection_is_collection_verified_supply", "assets_v3_delegate", "assets_v3_is_burnt", "assets_v3_is_compressed", @@ -203,7 +205,7 @@ impl PgClient { for (index, on_query_string) in [ ("assets_v3_authority_fk", "assets_v3(ast_authority_fk) WHERE ast_authority_fk IS NOT NULL"), - ("assets_v3_collection_is_collection_verified", "assets_v3(ast_collection, ast_is_collection_verified) WHERE ast_collection IS NOT NULL"), + ("assets_v3_collection_is_collection_verified_supply", "assets_v3(ast_collection, ast_is_collection_verified, ast_supply) WHERE ast_collection IS NOT NULL AND ast_supply IS NOT NULL"), ("assets_v3_delegate", "assets_v3(ast_delegate) WHERE ast_delegate IS NOT NULL"), ("assets_v3_is_burnt", "assets_v3(ast_is_burnt) WHERE ast_is_burnt IS TRUE"), ("assets_v3_is_compressed", "assets_v3(ast_is_compressed)"), From 6b1cb344319db9d77b4682fb568ad71020dbce36 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Mon, 3 Mar 2025 15:35:39 +0000 Subject: [PATCH 10/21] chore(ci): modularize release preparation process [MTG-1360] (#436) * refactor(release): modularize release preparation process - Extract inline shell scripts from GitHub Actions workflow into dedicated script files for better maintainability and testability - Fix cliff config allowing proper changelog creation * chore: Add newlines in scripts * chore(CI): inline most steps in release process * chore: Cleanup cliff.toml --- .github/workflows/release-prepare.yml | 109 +++++++++----------------- Makefile | 17 +++- cliff.toml | 44 +++++++---- scripts/release/prepare_changelog.sh | 90 +++++++++++++++++++++ scripts/update_changelog.sh | 100 +++++++++++++++++++++++ 5 files changed, 271 insertions(+), 89 deletions(-) create mode 100755 scripts/release/prepare_changelog.sh create mode 100755 scripts/update_changelog.sh diff --git a/.github/workflows/release-prepare.yml b/.github/workflows/release-prepare.yml index 4609e47aa..e93465ab5 100644 --- a/.github/workflows/release-prepare.yml +++ b/.github/workflows/release-prepare.yml @@ -35,32 +35,43 @@ jobs: ref: ${{ inputs.base_commit || 'develop' }} fetch-depth: 0 # Fetch all history for changelog generation - - name: Validate input version - run: | - set -e - # Make sure the version follows semantic versioning format - if ! echo "${{ inputs.version }}" | grep -E '^[0-9]+\.[0-9]+\.[0-9]+$'; then - echo "Error: Version must follow semantic versioning format (e.g., 0.5.0)" - exit 1 - fi - - - name: Setup Git Identity + - name: Setup environment run: | set -e + # Setup Git Identity git config --global user.name "GitHub Actions" git config --global user.email "actions@github.com" - - - name: Install dependencies - run: | - set -e + + # Install dependencies sudo apt-get update && sudo apt-get install -y protobuf-compiler cargo install cargo-edit cargo install git-cliff + - name: Validate version format + run: | + set -e + VERSION="${{ inputs.version }}" + + # Make sure version is not empty + if [ -z "$VERSION" ]; then + echo "Error: Version is required" + exit 1 + fi + + # Validate using semver regex pattern + if ! [[ "$VERSION" =~ ^[0-9]+\.[0-9]+\.[0-9]+$ ]]; then + echo "Error: Version '$VERSION' is not valid semantic versioning format." + echo "Expected format: MAJOR.MINOR.PATCH" + echo "Examples: 1.0.0, 0.5.0" + exit 1 + fi + + echo "Version format is valid: $VERSION" + - name: Create release branch run: | set -e - # Using standardized format: release/v{version} + # Create release branch with standardized format RELEASE_BRANCH="release/v${{ inputs.version }}" echo "Creating branch $RELEASE_BRANCH" git checkout -b $RELEASE_BRANCH @@ -72,7 +83,7 @@ jobs: # Update Cargo.toml versions find . -name "Cargo.toml" -type f -exec cargo set-version ${{ inputs.version }} --manifest-path {} \; - # Update any other version references (add any other files that contain version numbers) + # Update any other version references if [ -f "VERSION" ]; then echo "${{ inputs.version }}" > VERSION fi @@ -82,68 +93,24 @@ jobs: - name: Generate changelog id: changelog - run: | - set -e - # Check if cliff.toml exists - if [ ! -f "cliff.toml" ]; then - echo "Warning: cliff.toml not found, will use default configuration" - fi - - # Generate changelog using git-cliff - git-cliff --tag "v${{ inputs.version }}" --output CHANGELOG.md - - # Generate a shorter version for PR description - git-cliff --tag "v${{ inputs.version }}" --strip all > .changelog_content - - git add CHANGELOG.md - git commit -m "docs: add changelog for v${{ inputs.version }}" - - - name: Verify changelog - run: | - set -e - # Check that the changelog file exists and has content - if [ ! -s CHANGELOG.md ]; then - echo "Error: CHANGELOG.md is empty or does not exist" - exit 1 - fi - - # Check that the changelog contains the version we're releasing - if ! grep -q "v${{ inputs.version }}" CHANGELOG.md; then - echo "Error: CHANGELOG.md does not contain version v${{ inputs.version }}" - echo "Contents of CHANGELOG.md:" - cat CHANGELOG.md - exit 1 - fi - - # Check that the changelog has sections - if ! grep -q "###" CHANGELOG.md; then - echo "Warning: CHANGELOG.md does not contain any sections (###)" - echo "This might be ok if there are no conventional commits, but please verify" - fi - - echo "Changelog verification passed!" + run: ./scripts/release/prepare_changelog.sh "${{ inputs.version }}" - name: Push release branch run: | set -e - git push -u origin $RELEASE_BRANCH + git push -u origin ${{ env.RELEASE_BRANCH }} - - name: Create Pull Request - id: create-pr - uses: peter-evans/create-pull-request@v5 - with: - token: ${{ secrets.GITHUB_TOKEN }} - base: main - head: ${{ env.RELEASE_BRANCH }} - title: "Release v${{ inputs.version }}" - body-path: .changelog_content - draft: false - - - name: PR info + - name: Create pull request run: | set -e - echo "Pull Request created: ${{ steps.create-pr.outputs.pull-request-url }}" - echo "Please review the PR, make any necessary adjustments, and merge when ready." + gh pr create \ + --base main \ + --head ${{ env.RELEASE_BRANCH }} \ + --title "Release v${{ inputs.version }}" \ + --body-file .changelog_content \ + --draft false + + echo "Pull Request created. Please review the PR, make any necessary adjustments, and merge when ready." - name: Upload changelog artifact uses: actions/upload-artifact@v4 diff --git a/Makefile b/Makefile index 04ed81ce9..91ded16a1 100644 --- a/Makefile +++ b/Makefile @@ -1,4 +1,4 @@ -.PHONY: ci build start build-integrity-verification start-integrity-verification dev stop clippy test start-backfiller +.PHONY: ci build start build-integrity-verification start-integrity-verification dev stop clippy test start-backfiller ensure-git-cliff changelog release-changelog release SHELL := /bin/bash ci: @@ -52,6 +52,19 @@ test: ensure-git-cliff: @which git-cliff > /dev/null || cargo install git-cliff -# Generate a changelog using git-cliff +# Generate a full changelog using git-cliff changelog: @git-cliff --output CHANGELOG.md + +# Generate an incremental changelog for a specific release +# Usage: make release-changelog VERSION=0.5.0 [PREV_VERSION=v0.4.0] +release-changelog: + @scripts/update_changelog.sh "v$(VERSION)" $(PREV_VERSION) + +# Trigger release preparation workflow +# Usage: make release VERSION=0.5.0 [BASE_COMMIT=abc123] +release: + @echo "Preparing release v$(VERSION)..." + @[ -n "$(VERSION)" ] || (echo "Error: VERSION is required. Usage: make release VERSION=0.5.0"; exit 1) + @gh workflow run release-prepare.yml -f version=$(VERSION) $(if $(BASE_COMMIT),-f base_commit=$(BASE_COMMIT),) + @echo "Release preparation workflow triggered. Check GitHub Actions for progress." diff --git a/cliff.toml b/cliff.toml index c279ac1c4..01d29d6e0 100644 --- a/cliff.toml +++ b/cliff.toml @@ -27,7 +27,7 @@ body = """ {% for group, commits in commits | group_by(attribute="group") %} ### {{ group | upper_first }} {% for commit in commits %} -- {% if commit.breaking %}**BREAKING**: {% endif %}{{ commit.message | upper_first }} ([{{ commit.id | truncate(length=7, end="") }}](https://github.com/metaplex-foundation/aura/commit/{{ commit.id }})){% if commit.footers %} {% for footer in commit.footers %}{{ footer }}{% endfor %}{% endif %} +- {{ commit.message | upper_first }} ([{{ commit.id | truncate(length=7, end="") }}](https://github.com/metaplex-foundation/aura/commit/{{ commit.id }})) {% endfor %} {% endfor %}\n """ @@ -45,23 +45,37 @@ postprocessors = [ # parse the commits based on https://www.conventionalcommits.org conventional_commits = true # filter out the commits that are not conventional -filter_unconventional = true +filter_unconventional = false # process each line of a commit as an individual commit split_commits = false # regex for parsing and grouping commits commit_parsers = [ - { message = "^feat", group = "Features" }, - { message = "^fix", group = "Bug Fixes" }, - { message = "^doc", group = "Documentation" }, - { message = "^perf", group = "Performance" }, - { message = "^refactor", group = "Refactor" }, - { message = "^style", group = "Styling" }, - { message = "^test", group = "Testing" }, - { message = "^chore\\(release\\): prepare for", skip = true }, - { message = "^chore\\(deps\\)", group = "Dependencies" }, - { message = "^chore", group = "Miscellaneous Tasks" }, - { message = "^ci", group = "CI/CD" }, - { body = ".*security", group = "Security" }, + # Conventional commits with scope + { message = "^feat(\\(.*\\))?:", group = "Features" }, + { message = "^fix(\\(.*\\))?:", group = "Bug Fixes" }, + { message = "^docs(\\(.*\\))?:", group = "Documentation" }, + { message = "^doc(\\(.*\\))?:", group = "Documentation" }, + { message = "^perf(\\(.*\\))?:", group = "Performance" }, + { message = "^refactor(\\(.*\\))?:", group = "Refactor" }, + { message = "^style(\\(.*\\))?:", group = "Styling" }, + { message = "^test(\\(.*\\))?:", group = "Testing" }, + { message = "^chore\\(release\\):", skip = true }, + { message = "^chore\\(deps\\)(\\(.*\\))?:", group = "Dependencies" }, + { message = "^chore(\\(.*\\))?:", group = "Miscellaneous Tasks" }, + { message = "^ci(\\(.*\\))?:", group = "CI/CD" }, + + # Simple commits without scope + { message = "^feat\\s+", group = "Features" }, + { message = "^fix\\s+", group = "Bug Fixes" }, + { message = "^doc\\s+", group = "Documentation" }, + { message = "^test\\s+", group = "Testing" }, + + # Ticket prefixed commits + { message = "^MTG-\\d+\\s+.*(fix|fixed)", group = "Bug Fixes" }, + { message = "^MTG-\\d+\\s+.*(feat|feature)", group = "Features" }, + { message = "^MTG-\\d+\\s+.*(doc|docs)", group = "Documentation" }, + { message = "^MTG-\\d+\\s+.*(test)", group = "Testing" }, + { message = "^MTG-\\d+\\s+", group = "Tasks" }, ] # filter out the commits that are not matched by commit parsers filter_commits = false @@ -75,5 +89,3 @@ ignore_tags = "" date_order = false # sort the commits inside sections by oldest/newest order sort_commits = "oldest" -# limit the number of commits included in the changelog -limit_commits = 0 diff --git a/scripts/release/prepare_changelog.sh b/scripts/release/prepare_changelog.sh new file mode 100755 index 000000000..047e1b83e --- /dev/null +++ b/scripts/release/prepare_changelog.sh @@ -0,0 +1,90 @@ +#!/bin/bash +set -e + +# This script generates a changelog for a new version, verifies it, and extracts the section for PR +# Usage: prepare_changelog.sh + +VERSION=$1 + +if [ -z "$VERSION" ]; then + echo "Error: Version is required" + echo "Usage: $0 " + exit 1 +fi + +# Find the previous version tag +PREVIOUS_VERSION=$(git describe --tags --abbrev=0 2>/dev/null || echo "") +if [ -z "$PREVIOUS_VERSION" ]; then + echo "No previous tags found. Will generate full changelog." + PREVIOUS_VERSION="" +else + echo "Found previous version tag: $PREVIOUS_VERSION" + # Export for GitHub Actions if needed + if [ -n "$GITHUB_OUTPUT" ]; then + echo "previous_version=$PREVIOUS_VERSION" >> "$GITHUB_OUTPUT" + fi +fi + +# Generate changelog incrementally or in full +if [ -n "$PREVIOUS_VERSION" ]; then + echo "Generating incremental changelog from $PREVIOUS_VERSION to v$VERSION" + ./scripts/update_changelog.sh "v$VERSION" "$PREVIOUS_VERSION" +else + echo "Generating full changelog for v$VERSION" + git-cliff --config cliff.toml --tag "v$VERSION" --output CHANGELOG.md +fi + +# Verify the changelog +echo "Verifying changelog..." +# Check that the changelog file exists and has content +if [ ! -s "CHANGELOG.md" ]; then + echo "Error: CHANGELOG.md is empty or does not exist" + exit 1 +fi + +# Check that the changelog contains the version we're releasing +if ! grep -q "v$VERSION" CHANGELOG.md; then + echo "Error: CHANGELOG.md does not contain version v$VERSION" + echo "Contents of CHANGELOG.md:" + cat CHANGELOG.md + exit 1 +fi + +# Check that the changelog has sections +if ! grep -q "###" CHANGELOG.md; then + echo "Warning: CHANGELOG.md does not contain any sections (###)" + echo "This might be ok if there are no conventional commits, but please verify" +fi + +echo "Changelog verification passed!" + +# Extract changelog section for PR description +echo "Extracting changelog section for PR description..." +OUTPUT_FILE=".changelog_content" + +# Extract just the section for this version +awk 'BEGIN{section=0; found=0} + /^## \[.*\]/{ + if(!found) { + section=1; + found=1; + } else { + section=0; + } + } + section{print}' CHANGELOG.md | grep -v "^\[" > "$OUTPUT_FILE" + +# Check if we extracted any content +if [ ! -s "$OUTPUT_FILE" ]; then + echo "No content extracted from CHANGELOG.md. Check your changelog file." + echo "No changelog content found" > "$OUTPUT_FILE" + exit 1 +fi + +echo "Extracted changelog section to $OUTPUT_FILE" + +# Commit changes +git add CHANGELOG.md +git commit -m "docs: add changelog for v$VERSION" + +echo "Changelog for v$VERSION has been generated and committed" diff --git a/scripts/update_changelog.sh b/scripts/update_changelog.sh new file mode 100755 index 000000000..b117aae7c --- /dev/null +++ b/scripts/update_changelog.sh @@ -0,0 +1,100 @@ +#!/bin/bash +set -e + +# This script updates the CHANGELOG.md by appending new changes instead of regenerating it +# It takes two parameters: +# $1: New version tag (e.g., v0.5.0) +# $2: Previous version tag (optional, will be detected automatically if not provided) + +NEW_VERSION=$1 +PREVIOUS_VERSION=$2 + +if [ -z "$NEW_VERSION" ]; then + echo "Error: New version tag is required" + echo "Usage: $0 [previous_version_tag]" + exit 1 +fi + +# If previous version is not provided, find the latest tag +if [ -z "$PREVIOUS_VERSION" ]; then + PREVIOUS_VERSION=$(git describe --tags --abbrev=0 2>/dev/null || echo "") + + if [ -z "$PREVIOUS_VERSION" ]; then + # If no tags exist, use the first commit + PREVIOUS_VERSION=$(git rev-list --max-parents=0 HEAD) + echo "No previous tags found, using first commit: $PREVIOUS_VERSION" + else + echo "Using latest tag as previous version: $PREVIOUS_VERSION" + fi +fi + +# Define filenames +TEMP_CHANGELOG="CHANGELOG.temp.md" +FINAL_CHANGELOG="CHANGELOG.md" +HEADER_FILE="CHANGELOG.header.md" +NEW_CONTENT_FILE="CHANGELOG.new.md" + +# Check if existing changelog exists +if [ ! -f "$FINAL_CHANGELOG" ]; then + echo "No existing CHANGELOG.md found, will create a new one" + touch "$FINAL_CHANGELOG" +fi + +# Check if the new version tag exists, use HEAD if it doesn't +if ! git rev-parse "$NEW_VERSION" >/dev/null 2>&1; then + echo "Tag $NEW_VERSION doesn't exist yet, using HEAD for changelog generation" + RANGE_END="HEAD" +else + RANGE_END="$NEW_VERSION" +fi + +# Create a temporary changelog for just the new changes +if [ "$RANGE_END" = "HEAD" ]; then + echo "Generating changelog for $PREVIOUS_VERSION..HEAD with tag $NEW_VERSION" + git-cliff --config cliff.toml --tag "$NEW_VERSION" --output "$TEMP_CHANGELOG" "$PREVIOUS_VERSION..HEAD" +else + echo "Generating changelog for $PREVIOUS_VERSION..$NEW_VERSION" + git-cliff --config cliff.toml --tag "$NEW_VERSION" --output "$TEMP_CHANGELOG" "$PREVIOUS_VERSION..$NEW_VERSION" +fi + +# Extract the header (everything before the first version section) +awk 'BEGIN{header=1} /^## \[.*\]/{if(header) {header=0}} header{print}' "$TEMP_CHANGELOG" > "$HEADER_FILE" + +# Extract the new content (just the first version section) +awk 'BEGIN{section=0; found=0} + /^## \[.*\]/{ + if(!found) { + section=1; + found=1; + print; + } else { + section=0; + } + next; + } + section{print}' "$TEMP_CHANGELOG" > "$NEW_CONTENT_FILE" + +# Check if we got any new content +if [ ! -s "$NEW_CONTENT_FILE" ]; then + echo "No new content generated. Check your version tags and commit history." + rm -f "$TEMP_CHANGELOG" "$HEADER_FILE" "$NEW_CONTENT_FILE" + exit 1 +fi + +# Create the final changelog by combining header, new content, and existing content (minus the header) +{ + cat "$HEADER_FILE" + cat "$NEW_CONTENT_FILE" + if [ -s "$FINAL_CHANGELOG" ]; then + # Skip the header from the existing changelog + awk 'BEGIN{header=1} /^## \[.*\]/{if(header) {header=0; print; next}} !header{print}' "$FINAL_CHANGELOG" + fi +} > "$TEMP_CHANGELOG" + +# Move the temporary changelog to the final location +mv "$TEMP_CHANGELOG" "$FINAL_CHANGELOG" + +# Clean up temporary files +rm -f "$HEADER_FILE" "$NEW_CONTENT_FILE" + +echo "Updated CHANGELOG.md with changes from $PREVIOUS_VERSION to $NEW_VERSION" From 427cc6dc0c780d36b0a9a6c1c10f4bbb533dec03 Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Mon, 3 Mar 2025 17:36:47 +0200 Subject: [PATCH 11/21] feat: overhaul task spawning in the whole project (#433) * feat: overhaul task spawning in the whole project Overhaul task spawning in all modules. Remove usage of the `mutexed_tasks` concept, remove usage of a broadcast channel that was used for shutdown, create a static executor to save running tasks & exit gracefully. * feat(nft_ingester): spawn graceful shutdown as the first task in `main` Spawn graceful shutdown as the first tokio task in binaries that perform it, await it at the end. * feat(ingester,synchronizer): introduce more cancellation checks * chore: remove patch versions from the ecosystem crates * feat(ingester): replace some loops with `run_until_cancelled` --- Cargo.lock | 31 +- Cargo.toml | 8 +- backfill_rpc/src/rpc.rs | 4 +- .../src/bin/compressed_assets/main.rs | 95 +++-- consistency_check/src/bin/jsons/main.rs | 73 ++-- .../src/bin/regular_assets/main.rs | 49 ++- consistency_check/src/lib.rs | 4 +- integration_tests/Cargo.toml | 1 + integration_tests/src/account_update_tests.rs | 18 +- integration_tests/src/cnft_tests.rs | 44 +-- integration_tests/src/common.rs | 52 ++- .../src/general_scenario_tests.rs | 29 +- integration_tests/src/lib.rs | 10 +- integration_tests/src/mpl_core_tests.rs | 166 +++------ integration_tests/src/regular_nft_tests.rs | 151 +++----- integration_tests/src/synchronizer_tests.rs | 53 +-- integration_tests/src/token_tests.rs | 19 +- integrity_verification/src/diff_checker.rs | 34 +- integrity_verification/src/main.rs | 10 +- interface/Cargo.toml | 1 + interface/src/fork_cleaner.rs | 4 +- nft_ingester/benches/ingester_benchmark.rs | 1 - nft_ingester/benches/integrated_benchmark.rs | 2 +- .../benches/synchronizer_benchmark.rs | 6 +- nft_ingester/src/ack.rs | 18 +- nft_ingester/src/api/api_impl.rs | 55 +-- .../src/api/backfilling_state_consistency.rs | 43 +-- nft_ingester/src/api/builder.rs | 63 +--- nft_ingester/src/api/dapi/asset.rs | 8 +- nft_ingester/src/api/dapi/change_logs.rs | 2 +- nft_ingester/src/api/dapi/get_asset.rs | 6 - nft_ingester/src/api/dapi/get_asset_batch.rs | 6 - nft_ingester/src/api/dapi/search_assets.rs | 8 - nft_ingester/src/api/service.rs | 32 +- .../api/synchronization_state_consistency.rs | 29 +- nft_ingester/src/backfiller.rs | 12 +- .../src/batch_mint/batch_mint_persister.rs | 31 +- .../src/batch_mint/batch_mint_processor.rs | 23 +- nft_ingester/src/bin/api/main.rs | 134 ++++--- nft_ingester/src/bin/backfill/main.rs | 183 +++++----- .../src/bin/burnt_assets_ingester/main.rs | 9 +- nft_ingester/src/bin/dumper/main.rs | 89 ++--- nft_ingester/src/bin/ingester/main.rs | 344 +++++++++--------- nft_ingester/src/bin/migrator/main.rs | 147 ++++---- nft_ingester/src/bin/raw_backup/main.rs | 12 +- nft_ingester/src/bin/rocksdb_backup/main.rs | 5 - nft_ingester/src/bin/slot_checker/main.rs | 31 +- nft_ingester/src/bin/slot_persister/main.rs | 71 ++-- nft_ingester/src/bin/synchronizer/main.rs | 130 ++++--- .../src/bin/synchronizer_utils/main.rs | 4 - nft_ingester/src/cleaners/fork_cleaner.rs | 15 +- nft_ingester/src/gapfiller.rs | 32 +- ...x_syncronizer.rs => index_synchronizer.rs} | 200 ++++++---- nft_ingester/src/init.rs | 44 +-- nft_ingester/src/json_worker.rs | 33 +- nft_ingester/src/lib.rs | 2 +- .../mpl_core_fee_indexing_processor.rs | 27 +- .../src/processors/accounts_processor.rs | 60 ++- .../src/processors/transaction_processor.rs | 27 +- nft_ingester/src/rocks_db.rs | 14 +- nft_ingester/src/scheduler.rs | 10 +- nft_ingester/src/sequence_consistent.rs | 13 +- nft_ingester/tests/api_tests.rs | 298 +++++---------- nft_ingester/tests/asset_previews_tests.rs | 4 +- nft_ingester/tests/backfill_tests.rs | 6 +- nft_ingester/tests/batch_mint_test.rs | 49 +-- nft_ingester/tests/bubblegum_tests.rs | 22 +- nft_ingester/tests/clean_forks_test.rs | 15 +- nft_ingester/tests/decompress.rs | 64 +--- nft_ingester/tests/dump_tests.rs | 65 ++-- nft_ingester/tests/gapfiller_tests.rs | 26 +- nft_ingester/tests/process_accounts.rs | 8 +- nft_ingester/tests/scheduler_tests.rs | 5 +- .../tests/sequence_consistent_tests.rs | 4 +- rocks-db/Cargo.toml | 1 + rocks-db/src/bin/column_copier/main.rs | 12 +- rocks-db/src/bin/fork_detector/main.rs | 8 +- rocks-db/src/bin/leaf_checker/main.rs | 13 +- rocks-db/src/bin/migrate_slots_db/main.rs | 8 +- .../src/clients/asset_streaming_client.rs | 4 +- rocks-db/src/clients/dump_client.rs | 9 +- .../clients/raw_blocks_streaming_client.rs | 4 +- rocks-db/src/fork_cleaner.rs | 6 +- rocks-db/src/lib.rs | 44 +-- rocks-db/src/migrator.rs | 2 - rocks-db/src/storage_traits.rs | 13 +- .../tests/asset_streaming_client_tests.rs | 12 +- .../tests/batch_client_integration_tests.rs | 2 - rocks-db/tests/dump_tests.rs | 6 +- rocks-db/tests/migration_tests.rs | 11 +- rocks-db/tests/parameters_tests.rs | 4 +- rocks-db/tests/raw_block_tests.rs | 4 +- rocks-db/tests/signature_client_tests.rs | 10 +- rocks-db/tests/urls_to_download_test.rs | 8 +- tests/setup/Cargo.toml | 1 + tests/setup/src/lib.rs | 41 ++- tests/setup/src/rocks.rs | 12 +- tests/txn_forwarder/src/utils.rs | 4 +- usecase/Cargo.toml | 1 + usecase/src/executor.rs | 27 ++ usecase/src/graceful_stop.rs | 28 +- usecase/src/lib.rs | 1 + usecase/src/slots_collector.rs | 10 +- 103 files changed, 1579 insertions(+), 2130 deletions(-) rename nft_ingester/src/{index_syncronizer.rs => index_synchronizer.rs} (87%) create mode 100644 usecase/src/executor.rs diff --git a/Cargo.lock b/Cargo.lock index dec6008ed..13ff0005a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1676,7 +1676,7 @@ dependencies = [ "anchor-lang 0.29.0", "async-trait", "blockbuster", - "borsh 0.9.3", + "borsh 0.10.3", "bytemuck", "chrono", "mpl-bubblegum", @@ -2067,7 +2067,7 @@ dependencies = [ "memchr", "pin-project-lite", "tokio", - "tokio-util 0.7.11", + "tokio-util 0.7.13", ] [[package]] @@ -2107,7 +2107,7 @@ dependencies = [ "tempfile", "thiserror", "tokio", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tracing", "tracing-subscriber", "usecase", @@ -3344,7 +3344,7 @@ dependencies = [ "indexmap 2.2.6", "slab", "tokio", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tracing", ] @@ -3874,6 +3874,7 @@ dependencies = [ "tempfile", "tokio", "tokio-stream", + "tokio-util 0.7.13", "tracing", "tracing-test", "usecase", @@ -3907,7 +3908,7 @@ dependencies = [ "spl-concurrent-merkle-tree 0.2.0", "thiserror", "tokio", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tracing", "usecase", ] @@ -3934,6 +3935,7 @@ dependencies = [ "solana-transaction-status", "thiserror", "tokio", + "tokio-util 0.7.13", ] [[package]] @@ -4856,7 +4858,7 @@ dependencies = [ "thiserror", "tokio", "tokio-retry", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tonic 0.10.2", "tracing", "tracing-subscriber", @@ -6326,7 +6328,7 @@ dependencies = [ "sha1_smol", "tokio", "tokio-native-tls", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "url", ] @@ -6448,7 +6450,7 @@ dependencies = [ "tokio", "tokio-native-tls", "tokio-rustls 0.24.1", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tower-service", "url", "wasm-bindgen", @@ -6562,6 +6564,7 @@ dependencies = [ "thiserror", "tokio", "tokio-stream", + "tokio-util 0.7.13", "tracing", "tracing-test", "usecase", @@ -7136,6 +7139,7 @@ dependencies = [ "testcontainers", "testcontainers-modules", "tokio", + "tokio-util 0.7.13", "tracing", "uuid 1.8.0", ] @@ -9798,9 +9802,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.11" +version = "0.7.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9cf6b47b3771c49ac75ad09a6162f53ad4b8088b76ac60e8ec1455b31a189fe1" +checksum = "d7fcaa8d55a2bdd6b83ace262b016eca0d79ee02818c5c1bcdf0305114081078" dependencies = [ "bytes", "futures-core", @@ -9901,7 +9905,7 @@ dependencies = [ "tokio", "tokio-rustls 0.23.4", "tokio-stream", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tower", "tower-layer", "tower-service", @@ -10020,7 +10024,7 @@ dependencies = [ "rand 0.8.5", "slab", "tokio", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tower-layer", "tower-service", "tracing", @@ -10396,6 +10400,7 @@ dependencies = [ "spl-concurrent-merkle-tree 0.4.1", "thiserror", "tokio", + "tokio-util 0.7.13", "tracing", ] @@ -10522,7 +10527,7 @@ dependencies = [ "serde_urlencoded", "tokio", "tokio-tungstenite 0.21.0", - "tokio-util 0.7.11", + "tokio-util 0.7.13", "tower-service", "tracing", ] diff --git a/Cargo.toml b/Cargo.toml index b43cf6bd3..2fe4c0714 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,10 +18,10 @@ members = [ [workspace.dependencies] # Tokio ecosystem -tokio = { version = "1.37.0", features = ["full", "tracing"] } -tokio-stream = "0.1.11" -tokio-util = { version = "0.7.10", features = ["codec", "compat"] } -tokio-retry = "0.3.0" +tokio = { version = "1.37", features = ["full", "tracing"] } +tokio-stream = "0.1" +tokio-util = { version = "0.7", features = ["codec", "compat"] } +tokio-retry = "0.3" # Serde ecosystem and seryalization tools diff --git a/backfill_rpc/src/rpc.rs b/backfill_rpc/src/rpc.rs index 28b1390c2..1ea3797e0 100644 --- a/backfill_rpc/src/rpc.rs +++ b/backfill_rpc/src/rpc.rs @@ -346,7 +346,7 @@ mod tests { }); let (addr, server) = warp::serve(route).bind_ephemeral(([127, 0, 0, 1], 0)); - let _server_handle = tokio::spawn(server); + let _server_handle = usecase::executor::spawn(server); let client = RpcClient::new_sender(TooManyRequestsRpcSender(addr.to_string()), Default::default()); @@ -373,7 +373,7 @@ mod tests { }); let (addr, server) = warp::serve(route).bind_ephemeral(([127, 0, 0, 1], 0)); - let _server_handle = tokio::spawn(server); + let _server_handle = usecase::executor::spawn(server); let client = RpcClient::new_sender(TooManyRequestsRpcSender(addr.to_string()), Default::default()); diff --git a/consistency_check/src/bin/compressed_assets/main.rs b/consistency_check/src/bin/compressed_assets/main.rs index 02af5bcbf..651373e79 100644 --- a/consistency_check/src/bin/compressed_assets/main.rs +++ b/consistency_check/src/bin/compressed_assets/main.rs @@ -21,12 +21,12 @@ use solana_sdk::pubkey::{ParsePubkeyError, Pubkey}; use spl_concurrent_merkle_tree::hash::recompute; use tempfile::TempDir; use tokio::{ - sync::{broadcast, Mutex, OwnedSemaphorePermit, Semaphore}, - task::{JoinError, JoinSet}, + sync::{Mutex, OwnedSemaphorePermit, Semaphore}, + task::JoinError, }; use tokio_util::sync::CancellationToken; use tracing::{error, info}; -use usecase::{graceful_stop::graceful_stop, proofs::MaybeProofChecker}; +use usecase::proofs::MaybeProofChecker; #[derive(Parser, Debug)] #[command(author, version, about, long_about = None)] @@ -72,16 +72,11 @@ pub async fn main() { let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); let temp_dir = TempDir::new().expect("Failed to create temp directory"); let temp_path = temp_dir.path().to_path_buf(); + let cancellation_token = CancellationToken::new(); info!("Opening DB..."); let db_client = Arc::new( - Storage::open_secondary( - config.db_path, - temp_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - red_metrics, - MigrationState::Last, - ) - .unwrap(), + Storage::open_secondary(config.db_path, temp_path, red_metrics, MigrationState::Last) + .unwrap(), ); info!("DB opened"); @@ -100,12 +95,6 @@ pub async fn main() { let assets_with_failed_proofs = Arc::new(AtomicU64::new(0)); let assets_with_missed_proofs = Arc::new(AtomicU64::new(0)); - let shutdown_token = CancellationToken::new(); - let (shutdown_for_file_writer_tx, shutdown_for_file_writer_rx) = broadcast::channel::<()>(1); - - let mut tasks = JoinSet::new(); - let mut writers = JoinSet::new(); - let failed_proofs: Arc>>> = Arc::new(Mutex::new(HashMap::new())); let failed_check: Arc>> = Arc::new(Mutex::new(HashSet::new())); @@ -113,13 +102,13 @@ pub async fn main() { let chunk_size = keys.len().div_ceil(config.workers); for chunk in keys.chunks(chunk_size) { - tasks.spawn(verify_tree_batch( + usecase::executor::spawn(verify_tree_batch( progress_bar.clone(), assets_processed.clone(), assets_with_failed_proofs.clone(), assets_with_missed_proofs.clone(), rate.clone(), - shutdown_token.clone(), + cancellation_token.child_token(), chunk.to_vec(), rpc_client.clone(), db_client.clone(), @@ -132,45 +121,45 @@ pub async fn main() { let assets_processed_clone = assets_processed.clone(); let rate_clone = rate.clone(); - let shutdown_token_clone = shutdown_token.clone(); - // update rate on the background - tokio::spawn(update_rate(shutdown_token_clone, assets_processed_clone, rate_clone)); + usecase::executor::spawn(update_rate( + cancellation_token.child_token(), + assets_processed_clone, + rate_clone, + )); // write found problematic assets to the files - writers.spawn(async move { - let failed_checks_file = - File::create(FAILED_CHECKS_FILE).expect("Failed to create file for failed check trees"); - let failed_proofs_file = - File::create(FAILED_PROOFS_FILE).expect("Failed to create file for failed proofs"); - - let mut failed_checks_wrt = csv::Writer::from_writer(failed_checks_file); - let mut failed_proofs_wrt = csv::Writer::from_writer(failed_proofs_file); - - loop { - if let Err(e) = process_failed_checks(&mut failed_checks_wrt, &failed_check).await { - error!("Error writing failed checks: {}", e); - } + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + let failed_checks_file = File::create(FAILED_CHECKS_FILE) + .expect("Failed to create file for failed check trees"); + let failed_proofs_file = + File::create(FAILED_PROOFS_FILE).expect("Failed to create file for failed proofs"); + + let mut failed_checks_wrt = csv::Writer::from_writer(failed_checks_file); + let mut failed_proofs_wrt = csv::Writer::from_writer(failed_proofs_file); + + loop { + if let Err(e) = process_failed_checks(&mut failed_checks_wrt, &failed_check).await { + error!("Error writing failed checks: {}", e); + } - if let Err(e) = process_failed_proofs(&mut failed_proofs_wrt, &failed_proofs).await { - error!("Error writing failed proofs: {}", e); - } + if let Err(e) = process_failed_proofs(&mut failed_proofs_wrt, &failed_proofs).await + { + error!("Error writing failed proofs: {}", e); + } - if !shutdown_for_file_writer_rx.is_empty() { - break; - } + if cancellation_token.is_cancelled() { + break; + } - tokio::time::sleep(WRITER_SLEEP_TIME).await; + tokio::time::sleep(WRITER_SLEEP_TIME).await; + } } - - Ok(()) }); - graceful_stop(&mut tasks).await; - - shutdown_for_file_writer_tx.send(()).unwrap(); - - graceful_stop(&mut writers).await; + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; progress_bar.finish_with_message("Processing complete"); } @@ -182,7 +171,7 @@ async fn verify_tree_batch( assets_with_failed_proofs: Arc, assets_with_missed_proofs: Arc, rate: Arc>, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, trees: Vec, rpc: Arc, rocks: Arc, @@ -214,7 +203,7 @@ async fn verify_tree_batch( let mut assets_batch = Vec::new(); for asset_index in 0..des_data.num_minted { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Received shutdown signal"); return Ok(()); } @@ -234,7 +223,7 @@ async fn verify_tree_batch( let progress_bar_cloned = progress_bar.clone(); let permit = semaphore.clone().acquire_owned().await.unwrap(); - tokio::spawn(process_assets_batch( + usecase::executor::spawn(process_assets_batch( rocks_cloned, api_metrics_cloned, failed_proofs_cloned, @@ -262,7 +251,7 @@ async fn verify_tree_batch( let progress_bar_cloned = progress_bar.clone(); let permit = semaphore.clone().acquire_owned().await.unwrap(); - tokio::spawn(process_assets_batch( + usecase::executor::spawn(process_assets_batch( rocks_cloned, api_metrics_cloned, failed_proofs_cloned, diff --git a/consistency_check/src/bin/jsons/main.rs b/consistency_check/src/bin/jsons/main.rs index 926a47496..d21db156a 100644 --- a/consistency_check/src/bin/jsons/main.rs +++ b/consistency_check/src/bin/jsons/main.rs @@ -15,13 +15,9 @@ use indicatif::{ProgressBar, ProgressStyle}; use nft_ingester::init::init_index_storage_with_migration; use rocks_db::{migrator::MigrationState, Storage}; use tempfile::TempDir; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; +use tokio::sync::Mutex; use tokio_util::sync::CancellationToken; use tracing::{error, info}; -use usecase::graceful_stop::graceful_stop; const WRITER_SLEEP_TIME: Duration = Duration::from_secs(30); const SLEEP_AFTER_ERROR: Duration = Duration::from_secs(3); @@ -132,16 +128,11 @@ async fn check_jsons_consistency(rocks_path: String, postgre_creds: String, batc let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); let temp_dir = TempDir::new().expect("Failed to create temp directory"); let temp_path = temp_dir.path().to_str().unwrap().to_string(); + let cancellation_token = CancellationToken::new(); info!("Opening DB..."); let db_client = Arc::new( - Storage::open_secondary( - rocks_path, - temp_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .unwrap(), + Storage::open_secondary(rocks_path, temp_path, red_metrics.clone(), MigrationState::Last) + .unwrap(), ); info!("DB opened"); @@ -176,46 +167,45 @@ async fn check_jsons_consistency(rocks_path: String, postgre_creds: String, batc let rate = Arc::new(Mutex::new(0.0)); let count_of_missed_jsons = Arc::new(AtomicU64::new(0)); - let shutdown_token = CancellationToken::new(); - let (shutdown_for_file_writer_tx, shutdown_for_file_writer_rx) = broadcast::channel::<()>(1); - - let mut writers = JoinSet::new(); - let missed_jsons: Arc>> = Arc::new(Mutex::new(HashSet::new())); info!("Launching writer job..."); let missed_jsons_cloned = missed_jsons.clone(); - writers.spawn(async move { - let missed_jsons_file = - File::create(MISSED_JSONS_FILE).expect("Failed to create file for missed jsons"); + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + let missed_jsons_file = + File::create(MISSED_JSONS_FILE).expect("Failed to create file for missed jsons"); + + let mut missed_jsons_wrt = csv::Writer::from_writer(missed_jsons_file); + + loop { + let mut f_ch = missed_jsons_cloned.lock().await; + for t in f_ch.iter() { + missed_jsons_wrt.write_record([t]).unwrap(); + } + missed_jsons_wrt.flush().unwrap(); + f_ch.clear(); - let mut missed_jsons_wrt = csv::Writer::from_writer(missed_jsons_file); + drop(f_ch); - loop { - let mut f_ch = missed_jsons_cloned.lock().await; - for t in f_ch.iter() { - missed_jsons_wrt.write_record([t]).unwrap(); - } - missed_jsons_wrt.flush().unwrap(); - f_ch.clear(); - - drop(f_ch); + if cancellation_token.is_cancelled() { + break; + } - if !shutdown_for_file_writer_rx.is_empty() { - break; + tokio::time::sleep(WRITER_SLEEP_TIME).await; } - - tokio::time::sleep(WRITER_SLEEP_TIME).await; } - - Ok(()) }); info!("Launching rate updater..."); let assets_processed_clone = assets_processed.clone(); - let shutdown_token_clone = shutdown_token.clone(); let rate_clone = rate.clone(); - tokio::spawn(update_rate(shutdown_token_clone, assets_processed_clone, rate_clone)); + usecase::executor::spawn(update_rate( + cancellation_token.child_token(), + assets_processed_clone, + rate_clone, + )); let mut last_key_in_batch = None; @@ -297,9 +287,6 @@ async fn check_jsons_consistency(rocks_path: String, postgre_creds: String, batc )); } + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; info!("Main loop finished its job"); - - shutdown_for_file_writer_tx.send(()).unwrap(); - - graceful_stop(&mut writers).await; } diff --git a/consistency_check/src/bin/regular_assets/main.rs b/consistency_check/src/bin/regular_assets/main.rs index 842d30520..3161680b3 100644 --- a/consistency_check/src/bin/regular_assets/main.rs +++ b/consistency_check/src/bin/regular_assets/main.rs @@ -20,11 +20,10 @@ use solana_sdk::pubkey::Pubkey; use tempfile::TempDir; use tokio::{ sync::{mpsc::Receiver, Mutex, Semaphore}, - task::{JoinError, JoinSet}, + task::JoinError, }; use tokio_util::sync::CancellationToken; use tracing::{error, info}; -use usecase::graceful_stop::graceful_stop; mod snapshot_reader; @@ -78,21 +77,14 @@ pub async fn main() { let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); let temp_dir = TempDir::new().expect("Failed to create temp directory"); let temp_path = temp_dir.path().to_str().unwrap().to_string(); + let cancellation_token = CancellationToken::new(); info!("Opening DB..."); let db_client = Arc::new( - Storage::open_secondary( - config.db_path, - temp_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - red_metrics, - MigrationState::Last, - ) - .unwrap(), + Storage::open_secondary(config.db_path, temp_path, red_metrics, MigrationState::Last) + .unwrap(), ); info!("DB opened"); - let shutdown_token = CancellationToken::new(); - let spinner_style = ProgressStyle::with_template("{prefix:>10.bold.dim} {spinner} total={human_pos} {msg}") .unwrap(); @@ -114,12 +106,10 @@ pub async fn main() { let (fungibles_channel_tx, fungibles_channel_rx) = tokio::sync::mpsc::channel::(CHANNEL_SIZE); - let mut tasks = JoinSet::new(); - - tasks.spawn(process_nfts( + usecase::executor::spawn(process_nfts( config.inner_workers, db_client.clone(), - shutdown_token.clone(), + cancellation_token.child_token(), nfts_channel_rx, missed_asset_data.clone(), missed_mint_info.clone(), @@ -131,10 +121,10 @@ pub async fn main() { counter_missed_token.clone(), )); - tasks.spawn(process_fungibles( + usecase::executor::spawn(process_fungibles( config.inner_workers, db_client.clone(), - shutdown_token.clone(), + cancellation_token.child_token(), fungibles_channel_rx, missed_token_acc.clone(), assets_processed.clone(), @@ -151,15 +141,18 @@ pub async fn main() { info!("Snapshot file opened"); let assets_processed_clone = assets_processed.clone(); - let shutdown_token_clone = shutdown_token.clone(); let rate_clone = rate.clone(); - tokio::spawn(update_rate(shutdown_token_clone, assets_processed_clone, rate_clone)); + usecase::executor::spawn(update_rate( + cancellation_token.child_token(), + assets_processed_clone, + rate_clone, + )); 'outer: for append_vec in snapshot_loader.iter() { match append_vec { Ok(v) => { for account in append_vec_iter(Rc::new(v)) { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { break 'outer; } @@ -199,7 +192,7 @@ pub async fn main() { drop(nfts_channel_tx); drop(fungibles_channel_tx); - graceful_stop(&mut tasks).await; + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; if let Err(e) = write_data_to_file(config.missed_asset_file, &missed_asset_data).await { error!("Could not save keys with missed asset data: {}", e); @@ -218,7 +211,7 @@ pub async fn main() { async fn process_nfts( inner_workers: usize, rocks_db: Arc, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, mut nfts_channel_rx: Receiver<(AccountType, Pubkey)>, missed_asset_data: Arc>>, missed_mint_info: Arc>>, @@ -233,7 +226,7 @@ async fn process_nfts( let semaphore = Arc::new(Semaphore::new(inner_workers)); loop { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { break; } @@ -252,7 +245,7 @@ async fn process_nfts( let counter_missed_mint_cloned = counter_missed_mint.clone(); let counter_missed_token_cloned = counter_missed_token.clone(); - tokio::spawn(async move { + usecase::executor::spawn(async move { match rocks_db_cloned.asset_data.has_key(key).await { Ok(exist) => { if !exist { @@ -327,7 +320,7 @@ async fn process_nfts( async fn process_fungibles( inner_workers: usize, rocks_db: Arc, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, mut fungibles_channel_rx: Receiver, missed_token_acc: Arc>>, assets_processed: Arc, @@ -341,7 +334,7 @@ async fn process_fungibles( let semaphore = Arc::new(Semaphore::new(inner_workers)); loop { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { break; } @@ -359,7 +352,7 @@ async fn process_fungibles( let counter_missed_mint_cloned = counter_missed_mint.clone(); let counter_missed_token_cloned = counter_missed_token.clone(); - tokio::spawn(async move { + usecase::executor::spawn(async move { match rocks_db_cloned.token_accounts.has_key(key).await { Ok(exist) => { if !exist { diff --git a/consistency_check/src/lib.rs b/consistency_check/src/lib.rs index 55141fe6d..afc43b92b 100644 --- a/consistency_check/src/lib.rs +++ b/consistency_check/src/lib.rs @@ -7,7 +7,7 @@ use tokio::sync::Mutex; use tokio_util::sync::CancellationToken; pub async fn update_rate( - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, assets_processed: Arc, rate: Arc>, ) { @@ -19,7 +19,7 @@ pub async fn update_rate( tokio::select! { _ = sleep => {} - _ = shutdown_token.cancelled() => {break;} + _ = cancellation_token.cancelled() => { break; } } let current_time = std::time::Instant::now(); diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index 8601d5b29..a592fec38 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -27,6 +27,7 @@ spl-token = { workspace = true, features = ["no-entrypoint"] } sqlx = { workspace = true } tokio = { workspace = true } tokio-stream = { workspace = true } +tokio-util = { workspace = true } rocksdb = { workspace = true } tempfile = {workspace = true} env_logger = { workspace = true } diff --git a/integration_tests/src/account_update_tests.rs b/integration_tests/src/account_update_tests.rs index aec95514f..ccb13df08 100644 --- a/integration_tests/src/account_update_tests.rs +++ b/integration_tests/src/account_update_tests.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use borsh::BorshSerialize; use entities::api_req_params::{GetAsset, Options}; @@ -10,7 +10,6 @@ use plerkle_serialization::{ use serial_test::serial; use solana_sdk::{program_option::COption, program_pack::Pack, pubkey::Pubkey}; use spl_token::state::{Account as TokenAccount, AccountState, Mint}; -use tokio::{sync::Mutex, task::JoinSet}; use super::common::*; @@ -109,7 +108,7 @@ async fn index_account_update(setup: &TestSetup, pubkey: Pubkey, update: Account index_and_sync_account_bytes(setup, fbb.finished_data().to_vec()).await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_account_updates() { @@ -166,8 +165,6 @@ async fn test_account_updates() { }; let named_updates = vec![token_updated, mint_updated, metadata_updated]; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - // Test that stale updates are rejected and new updates are accepted for named_update in named_updates.clone() { if let AccountUpdate::None = named_update.update { @@ -178,8 +175,7 @@ async fn test_account_updates() { index_nft_accounts(&setup, get_nft_accounts(&setup, mint).await).await; - let response = - setup.das_api.get_asset(request.clone(), mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request.clone()).await.unwrap(); insta::assert_json_snapshot!(name.clone(), response); index_account_update( @@ -190,8 +186,7 @@ async fn test_account_updates() { ) .await; - let response_stale_lot = - setup.das_api.get_asset(request.clone(), mutexed_tasks.clone()).await.unwrap(); + let response_stale_lot = setup.das_api.get_asset(request.clone()).await.unwrap(); assert_eq!( response, response_stale_lot, "Update for {} account was not rejected", @@ -206,8 +201,7 @@ async fn test_account_updates() { ) .await; - let response_new_slot = - setup.das_api.get_asset(request.clone(), mutexed_tasks.clone()).await.unwrap(); + let response_new_slot = setup.das_api.get_asset(request.clone()).await.unwrap(); assert_ne!(response, response_new_slot); insta::assert_json_snapshot!( @@ -243,7 +237,7 @@ async fn test_account_updates() { } insta::assert_json_snapshot!( format!("{}-with-all-updates", name), - setup.das_api.get_asset(request.clone(), mutexed_tasks.clone()).await.unwrap() + setup.das_api.get_asset(request.clone()).await.unwrap() ); } } diff --git a/integration_tests/src/cnft_tests.rs b/integration_tests/src/cnft_tests.rs index 8bda76aa0..3772ea203 100644 --- a/integration_tests/src/cnft_tests.rs +++ b/integration_tests/src/cnft_tests.rs @@ -1,11 +1,10 @@ -use std::{collections::HashMap, str::FromStr, sync::Arc}; +use std::{collections::HashMap, str::FromStr}; use entities::api_req_params::{GetAsset, Options, SearchAssets}; use function_name::named; use itertools::Itertools; use serial_test::serial; use solana_sdk::signature::Signature; -use tokio::{sync::Mutex, task::JoinSet}; use super::common::*; @@ -23,18 +22,16 @@ pub async fn run_get_asset_scenario_test( Order::Forward => vec![seeds.iter().collect_vec()], }; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - for events in seed_permutations { index_seed_events(setup, events).await; let request = GetAsset { id: asset_id.to_string(), options: options.clone() }; - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(setup.name.clone(), response); } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] #[serial] #[named] @@ -76,7 +73,7 @@ async fn test_asset_decompress() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_cnft_scenario_mint_update_metadata() { @@ -110,7 +107,7 @@ async fn test_cnft_scenario_mint_update_metadata() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_cnft_scenario_mint_update_metadata_remove_creators() { @@ -146,7 +143,7 @@ async fn test_cnft_scenario_mint_update_metadata_remove_creators() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_cnft_owners_table() { @@ -170,8 +167,6 @@ async fn test_cnft_owners_table() { index_transaction(&setup, Signature::from_str(txn).unwrap()).await; } - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - for (request, individual_test_name) in [ ( SearchAssets { @@ -192,13 +187,12 @@ async fn test_cnft_owners_table() { "with_different_owner", ), ] { - let response = - setup.das_api.search_assets(request.clone(), mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request.clone()).await.unwrap(); insta::assert_json_snapshot!(format!("{}-{}", name, individual_test_name), response); } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_no_json_uri() { @@ -225,7 +219,7 @@ async fn test_mint_no_json_uri() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_delegate_transfer() { @@ -258,7 +252,7 @@ async fn test_mint_delegate_transfer() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_redeem_cancel_redeem() { @@ -291,7 +285,7 @@ async fn test_mint_redeem_cancel_redeem() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_redeem() { @@ -329,7 +323,7 @@ async fn test_mint_redeem() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_transfer_burn() { @@ -362,7 +356,7 @@ async fn test_mint_transfer_burn() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_transfer_noop() { @@ -395,7 +389,7 @@ async fn test_mint_transfer_noop() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_transfer_transfer() { @@ -428,7 +422,7 @@ async fn test_mint_transfer_transfer() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_verify_creator() { @@ -460,7 +454,7 @@ async fn test_mint_verify_creator() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_verify_collection() { @@ -492,7 +486,7 @@ async fn test_mint_verify_collection() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_transfer_mpl_programs() { @@ -525,7 +519,7 @@ async fn test_mint_transfer_mpl_programs() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_to_collection_unverify_collection() { @@ -562,7 +556,7 @@ async fn test_mint_to_collection_unverify_collection() { .await; } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mint_verify_collection_unverify_collection() { diff --git a/integration_tests/src/common.rs b/integration_tests/src/common.rs index 9fc5b5fdb..b1e820ef8 100644 --- a/integration_tests/src/common.rs +++ b/integration_tests/src/common.rs @@ -14,7 +14,7 @@ use nft_ingester::{ api::{account_balance::AccountBalanceGetterImpl, DasApi}, buffer::Buffer, config::JsonMiddlewareConfig, - index_syncronizer::Synchronizer, + index_synchronizer::Synchronizer, init::init_index_storage_with_migration, json_worker::JsonWorker, message_parser::MessageParser, @@ -46,11 +46,8 @@ use solana_sdk::{ signature::Signature, }; use solana_transaction_status::{EncodedConfirmedTransactionWithStatusMeta, UiTransactionEncoding}; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, - time::{sleep, Instant}, -}; +use tokio::time::{sleep, Instant}; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use usecase::proofs::MaybeProofChecker; @@ -99,10 +96,6 @@ pub struct TestSetup { } impl TestSetup { - pub async fn new(name: String) -> Self { - Self::new_with_options(name, TestSetupOptions::default()).await - } - pub async fn new_with_options(name: String, opts: TestSetupOptions) -> Self { let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); @@ -130,23 +123,18 @@ impl TestSetup { }; let client = Arc::new(RpcClient::new(rpc_url.to_string())); - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - let buffer = Arc::new(Buffer::new()); let metrics_state = MetricState::new(); - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let acc_processor = AccountsProcessor::build( - shutdown_rx.resubscribe(), + CancellationToken::new(), ACC_PROCESSOR_FETCH_BATCH_SIZE, buffer.clone(), metrics_state.ingester_metrics.clone(), None, index_storage.clone(), client.clone(), - mutexed_tasks.clone(), None, opts.well_known_fungible_accounts, ) @@ -160,13 +148,7 @@ impl TestSetup { } let storage = Arc::new( - Storage::open( - rocks_db_dir.path(), - mutexed_tasks.clone(), - red_metrics.clone(), - MigrationState::Last, - ) - .unwrap(), + Storage::open(rocks_db_dir.path(), red_metrics.clone(), MigrationState::Last).unwrap(), ); let tx_processor = @@ -412,10 +394,18 @@ pub async fn get_token_largest_account(client: &RpcClient, mint: Pubkey) -> anyh pub async fn index_and_sync_account_bytes(setup: &TestSetup, account_bytes: Vec) { process_and_save_accounts_to_rocks(setup, account_bytes).await; - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); + let cancellation_token = CancellationToken::new(); // copy data to Postgre - setup.synchronizer.synchronize_nft_asset_indexes(&shutdown_rx, 1000).await.unwrap(); - setup.synchronizer.synchronize_fungible_asset_indexes(&shutdown_rx, 1000).await.unwrap(); + setup + .synchronizer + .synchronize_nft_asset_indexes(cancellation_token.child_token(), 1000) + .await + .unwrap(); + setup + .synchronizer + .synchronize_fungible_asset_indexes(cancellation_token.child_token(), 1000) + .await + .unwrap(); } async fn process_and_save_accounts_to_rocks(setup: &TestSetup, account_bytes: Vec) { @@ -512,10 +502,13 @@ pub async fn index_transaction(setup: &TestSetup, sig: Signature) { .await .unwrap(); - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup.synchronizer.synchronize_nft_asset_indexes(&shutdown_rx, 1000).await.unwrap(); + setup.synchronizer.synchronize_nft_asset_indexes(CancellationToken::new(), 1000).await.unwrap(); - setup.synchronizer.synchronize_fungible_asset_indexes(&shutdown_rx, 1000).await.unwrap(); + setup + .synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 1000) + .await + .unwrap(); } async fn cached_fetch_largest_token_account_id(client: &RpcClient, mint: Pubkey) -> Pubkey { @@ -550,6 +543,7 @@ pub enum Network { Mainnet, Devnet, EclipseMainnet, + #[allow(unused)] EclipseDevnet, } diff --git a/integration_tests/src/general_scenario_tests.rs b/integration_tests/src/general_scenario_tests.rs index e7b457b66..1a99997f0 100644 --- a/integration_tests/src/general_scenario_tests.rs +++ b/integration_tests/src/general_scenario_tests.rs @@ -1,21 +1,16 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; -use entities::{ - api_req_params::{GetAsset, GetAssetsByOwner, SearchAssets}, - enums::Interface, -}; +use entities::api_req_params::{GetAsset, GetAssetsByOwner}; use function_name::named; use itertools::Itertools; -use nft_ingester::api::dapi::response::AssetList; use serial_test::serial; -use tokio::{sync::Mutex, task::JoinSet}; use crate::common::{ - index_seed_events, seed_accounts, seed_nfts, trim_test_name, well_known_fungible_tokens, - Network, SeedEvent, TestSetup, TestSetupOptions, + index_seed_events, seed_accounts, seed_nfts, trim_test_name, Network, SeedEvent, TestSetup, + TestSetupOptions, }; -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_asset_parsing() { @@ -40,14 +35,12 @@ async fn test_asset_parsing() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_get_different_assets_by_owner() { @@ -88,10 +81,8 @@ async fn test_get_different_assets_by_owner() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByOwner = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_owner(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_owner(request).await.unwrap(); insta::assert_json_snapshot!(name.clone(), response); let request = r#" @@ -110,13 +101,13 @@ async fn test_get_different_assets_by_owner() { "#; let request: GetAssetsByOwner = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_owner(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_owner(request).await.unwrap(); insta::assert_json_snapshot!(format!("{}_show_unverif_coll", name), response); } // context: this account had an issue with mpl-core < 0.9.0, where // it could not be parsed because of a bug in mpl-core. -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[named] async fn test_process_previously_unparseable_account() { let name = trim_test_name(function_name!()); diff --git a/integration_tests/src/lib.rs b/integration_tests/src/lib.rs index dd339e420..3c9b40717 100644 --- a/integration_tests/src/lib.rs +++ b/integration_tests/src/lib.rs @@ -1,10 +1,16 @@ -#![allow(unused)] - +#[cfg(test)] mod account_update_tests; +#[cfg(test)] mod cnft_tests; +#[cfg(test)] mod common; +#[cfg(test)] mod general_scenario_tests; +#[cfg(test)] mod mpl_core_tests; +#[cfg(test)] mod regular_nft_tests; +#[cfg(test)] mod synchronizer_tests; +#[cfg(test)] mod token_tests; diff --git a/integration_tests/src/mpl_core_tests.rs b/integration_tests/src/mpl_core_tests.rs index 9e2e7ef9c..9d897fd84 100644 --- a/integration_tests/src/mpl_core_tests.rs +++ b/integration_tests/src/mpl_core_tests.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use entities::api_req_params::{ GetAsset, GetAssetsByAuthority, GetAssetsByGroup, GetAssetsByOwner, @@ -6,11 +6,10 @@ use entities::api_req_params::{ use function_name::named; use itertools::Itertools; use serial_test::serial; -use tokio::{sync::Mutex, task::JoinSet}; use super::common::*; -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset() { @@ -35,14 +34,12 @@ async fn test_mpl_core_get_asset() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_collection() { @@ -67,14 +64,12 @@ async fn test_mpl_core_get_collection() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_assets_by_authority() { @@ -110,15 +105,12 @@ async fn test_mpl_core_get_assets_by_authority() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByAuthority = serde_json::from_str(request).unwrap(); - let response = - setup.das_api.get_assets_by_authority(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_authority(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_assets_by_group() { @@ -155,14 +147,12 @@ async fn test_mpl_core_get_assets_by_group() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByGroup = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_group(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_group(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_assets_by_owner() { @@ -196,14 +186,12 @@ async fn test_mpl_core_get_assets_by_owner() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByOwner = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_owner(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_owner(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_edition() { @@ -228,14 +216,12 @@ async fn test_mpl_core_get_asset_with_edition() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_pubkey_in_rule_set() { @@ -260,14 +246,12 @@ async fn test_mpl_core_get_asset_with_pubkey_in_rule_set() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_two_oracle_external_plugins() { @@ -292,14 +276,12 @@ async fn test_mpl_core_get_asset_with_two_oracle_external_plugins() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_oracle_external_plugin_on_collection() { @@ -324,14 +306,12 @@ async fn test_mpl_core_get_asset_with_oracle_external_plugin_on_collection() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_oracle_multiple_lifecycle_events() { @@ -356,14 +336,12 @@ async fn test_mpl_core_get_asset_with_oracle_multiple_lifecycle_events() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_oracle_custom_offset_and_base_address_config() { @@ -388,14 +366,12 @@ async fn test_mpl_core_get_asset_with_oracle_custom_offset_and_base_address_conf } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_oracle_no_offset() { @@ -420,14 +396,12 @@ async fn test_mpl_core_get_asset_with_oracle_no_offset() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_assets_by_group_with_oracle_and_custom_pda_all_seeds() { @@ -462,14 +436,12 @@ async fn test_mpl_core_get_assets_by_group_with_oracle_and_custom_pda_all_seeds( } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByGroup = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_group(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_group(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_multiple_internal_and_external_plugins() { @@ -494,14 +466,12 @@ async fn test_mpl_core_get_asset_with_multiple_internal_and_external_plugins() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_autograph_plugin() { @@ -526,14 +496,12 @@ async fn test_mpl_core_autograph_plugin() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_autograph_plugin_with_signature() { @@ -558,14 +526,12 @@ async fn test_mpl_core_autograph_plugin_with_signature() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_verified_creators_plugin() { @@ -590,14 +556,12 @@ async fn test_mpl_core_verified_creators_plugin() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_verified_creators_plugin_with_signature() { @@ -622,14 +586,12 @@ async fn test_mpl_core_verified_creators_plugin_with_signature() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_app_data_with_binary_data_and_owner_is_data_authority() { @@ -654,14 +616,12 @@ async fn test_mpl_core_get_asset_with_app_data_with_binary_data_and_owner_is_dat } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_app_data_with_json_data_and_update_authority_is_data_authority( @@ -687,14 +647,12 @@ async fn test_mpl_core_get_asset_with_app_data_with_json_data_and_update_authori } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_app_data_with_msg_pack_data_and_address_is_data_authority() { @@ -719,14 +677,12 @@ async fn test_mpl_core_get_asset_with_app_data_with_msg_pack_data_and_address_is } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_collection_with_linked_app_data_with_binary_data_and_address_is_data_authority( @@ -752,14 +708,12 @@ async fn test_mpl_core_get_collection_with_linked_app_data_with_binary_data_and_ } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_data_section_with_binary_data() { @@ -784,14 +738,12 @@ async fn test_mpl_core_get_asset_with_data_section_with_binary_data() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_collection_with_linked_app_data_with_json_data_and_owner_is_data_authority( @@ -817,14 +769,12 @@ async fn test_mpl_core_get_collection_with_linked_app_data_with_json_data_and_ow } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_data_section_with_json_data() { @@ -849,14 +799,12 @@ async fn test_mpl_core_get_asset_with_data_section_with_json_data() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_collection_with_linked_app_data_with_msg_pack_data_and_update_authority_is_data_authority( @@ -882,14 +830,12 @@ async fn test_mpl_core_get_collection_with_linked_app_data_with_msg_pack_data_an } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_mpl_core_get_asset_with_data_section_with_msg_pack_data() { @@ -914,9 +860,7 @@ async fn test_mpl_core_get_asset_with_data_section_with_msg_pack_data() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } diff --git a/integration_tests/src/regular_nft_tests.rs b/integration_tests/src/regular_nft_tests.rs index 31d2f5a38..994b4a6e9 100644 --- a/integration_tests/src/regular_nft_tests.rs +++ b/integration_tests/src/regular_nft_tests.rs @@ -1,4 +1,4 @@ -use std::{collections::HashMap, ptr::with_exposed_provenance, sync::Arc}; +use std::collections::HashMap; use entities::{ api_req_params::{GetAsset, GetAssetBatch, GetAssetsByGroup, SearchAssets}, @@ -6,25 +6,19 @@ use entities::{ }; use function_name::named; use itertools::Itertools; -use metrics_utils::IngesterMetricsConfig; use nft_ingester::{ api::dapi::response::AssetList, consts::RAYDIUM_API_HOST, raydium_price_fetcher::{RaydiumTokenPriceFetcher, CACHE_TTL}, scheduler::{update_fungible_token_static_details, Scheduler}, }; -use rocks_db::{batch_savers::BatchSaveStorage, storage_traits::AssetIndexReader}; use serial_test::serial; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; -use tracing::{info, warn}; +use tokio_util::sync::CancellationToken; use AssetType::NonFungible; use super::common::*; -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_reg_get_asset() { @@ -49,14 +43,12 @@ async fn test_reg_get_asset() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_reg_get_asset_batch() { @@ -109,15 +101,13 @@ async fn test_reg_get_asset_batch() { "2-and-a-missing-1", ), ] { - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetBatch = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset_batch(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset_batch(request).await.unwrap(); insta::assert_json_snapshot!(format!("{}-{}", name, individual_test_name), response); } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_reg_get_asset_by_group() { @@ -153,14 +143,12 @@ async fn test_reg_get_asset_by_group() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByGroup = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_group(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_group(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_reg_search_assets() { @@ -190,14 +178,12 @@ async fn test_reg_search_assets() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_regular_nft_collection() { @@ -222,14 +208,12 @@ async fn test_regular_nft_collection() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name.clone(), response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_search_by_owner_with_show_zero_balance() { @@ -278,10 +262,8 @@ async fn test_search_by_owner_with_show_zero_balance() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); // API shouldn't return zero NonFungible accounts ("3rzjtWZc"). "showZeroBalance": true is working only for Fungible tokens @@ -315,7 +297,7 @@ async fn test_search_by_owner_with_show_zero_balance() { "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -340,7 +322,7 @@ async fn test_search_by_owner_with_show_zero_balance() { "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -357,7 +339,7 @@ async fn test_search_by_owner_with_show_zero_balance() { insta::assert_json_snapshot!(format!("{}_token_type_fungible", name), response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_search_by_owner_with_show_zero_balance_false() { @@ -406,10 +388,8 @@ async fn test_search_by_owner_with_show_zero_balance_false() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -438,7 +418,7 @@ async fn test_search_by_owner_with_show_zero_balance_false() { "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -463,7 +443,7 @@ async fn test_search_by_owner_with_show_zero_balance_false() { "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -476,7 +456,7 @@ async fn test_search_by_owner_with_show_zero_balance_false() { insta::assert_json_snapshot!(format!("{}_token_type_fungible", name), response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_search_by_owner_with_show_zero_balance_with_reverse_data_processing_sequence() { @@ -525,10 +505,8 @@ async fn test_search_by_owner_with_show_zero_balance_with_reverse_data_processin } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); // API shouldn't return zero NonFungible accounts ("3rzjtWZc"). "showZeroBalance": true is working only for Fungible tokens @@ -562,7 +540,7 @@ async fn test_search_by_owner_with_show_zero_balance_with_reverse_data_processin "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -587,7 +565,7 @@ async fn test_search_by_owner_with_show_zero_balance_with_reverse_data_processin "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -604,7 +582,7 @@ async fn test_search_by_owner_with_show_zero_balance_with_reverse_data_processin insta::assert_json_snapshot!(format!("{}_token_type_fungible", name), response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_search_by_owner_with_show_zero_balance_false_with_reverse_data_processing_sequence() { @@ -653,10 +631,8 @@ async fn test_search_by_owner_with_show_zero_balance_false_with_reverse_data_pro } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -685,7 +661,7 @@ async fn test_search_by_owner_with_show_zero_balance_false_with_reverse_data_pro "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -710,7 +686,7 @@ async fn test_search_by_owner_with_show_zero_balance_false_with_reverse_data_pro "#; let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let res_obj: AssetList = serde_json::from_value(response.clone()).unwrap(); assert_eq!(res_obj.items.is_empty(), false); @@ -723,7 +699,7 @@ async fn test_search_by_owner_with_show_zero_balance_false_with_reverse_data_pro insta::assert_json_snapshot!(format!("{}_token_type_fungible", name), response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_search_assets_by_owner_with_pages() { @@ -769,10 +745,8 @@ async fn test_search_assets_by_owner_with_pages() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let name_all_assets = format!("{}_all_assets", test_name); @@ -792,10 +766,8 @@ async fn test_search_assets_by_owner_with_pages() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let name_page_1 = format!("{}_page_1", test_name); @@ -815,10 +787,8 @@ async fn test_search_assets_by_owner_with_pages() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let name_page_2 = format!("{}_page_2", test_name); @@ -838,17 +808,15 @@ async fn test_search_assets_by_owner_with_pages() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); let name_page_3 = format!("{}_page_3", test_name); insta::assert_json_snapshot!(name_page_3, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn get_asset_nft_token_22_with_metadata() { @@ -873,15 +841,13 @@ async fn get_asset_nft_token_22_with_metadata() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_requested_non_fungibles_are_non_fungibles() { @@ -931,10 +897,8 @@ async fn test_requested_non_fungibles_are_non_fungibles() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); assert_eq!(response["items"].as_array().unwrap().len(), 5); @@ -947,7 +911,7 @@ async fn test_requested_non_fungibles_are_non_fungibles() { insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_requested_fungibles_are_fungibles() { @@ -990,10 +954,8 @@ async fn test_requested_fungibles_are_fungibles() { } }"#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: SearchAssets = serde_json::from_str(request).unwrap(); - let response = setup.das_api.search_assets(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.search_assets(request).await.unwrap(); assert_eq!(response["items"].as_array().unwrap().len(), 1); @@ -1006,12 +968,11 @@ async fn test_requested_fungibles_are_fungibles() { insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_recognise_popular_fungible_tokens() { let name = trim_test_name(function_name!()); - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let token_price_fetcher = RaydiumTokenPriceFetcher::new(RAYDIUM_API_HOST.to_string(), CACHE_TTL, None); @@ -1045,7 +1006,7 @@ async fn test_recognise_popular_fungible_tokens() { }"#; let request = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_asset_batch(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset_batch(request).await.unwrap(); assert_eq!(response.as_array().unwrap().len(), 2); response.as_array().unwrap().iter().all(|i| { @@ -1056,12 +1017,11 @@ async fn test_recognise_popular_fungible_tokens() { insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_update_well_known_fungible_tokens() { let name = trim_test_name(function_name!()); - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let token_price_fetcher = RaydiumTokenPriceFetcher::new(RAYDIUM_API_HOST.to_string(), CACHE_TTL, None); @@ -1095,7 +1055,7 @@ async fn test_update_well_known_fungible_tokens() { }"#; let request = serde_json::from_str(request_str).unwrap(); - let response = setup.das_api.get_asset_batch(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset_batch(request).await.unwrap(); assert_eq!(response.as_array().unwrap().len(), 2); response.as_array().unwrap().iter().all(|i| { @@ -1104,18 +1064,16 @@ async fn test_update_well_known_fungible_tokens() { true }); - update_fungible_token_static_details( + let _ = update_fungible_token_static_details( &setup.rocks_db, well_known_fungible_accounts.keys().cloned().collect(), ); - // sync data to Postgre - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup.synchronizer.full_syncronize(&shutdown_rx, Fungible).await.unwrap(); - setup.synchronizer.full_syncronize(&shutdown_rx, NonFungible).await.unwrap(); + setup.synchronizer.full_syncronize(CancellationToken::new(), Fungible).await.unwrap(); + setup.synchronizer.full_syncronize(CancellationToken::new(), NonFungible).await.unwrap(); let request = serde_json::from_str(request_str).unwrap(); - let response = setup.das_api.get_asset_batch(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset_batch(request).await.unwrap(); assert_eq!(response.as_array().unwrap().len(), 2); response.as_array().unwrap().iter().all(|i| { @@ -1129,11 +1087,10 @@ async fn test_update_well_known_fungible_tokens() { #[named] #[serial] -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_update_fungible_token_static_details_job() { let name = trim_test_name(function_name!()); - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let setup = TestSetup::new_with_options( name.clone(), @@ -1161,7 +1118,7 @@ async fn test_update_fungible_token_static_details_job() { }"#; let request = serde_json::from_str(request_str).unwrap(); - let response = setup.das_api.get_asset_batch(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_asset_batch(request).await.unwrap(); assert_eq!(response.as_array().unwrap().len(), 2); response.as_array().unwrap().iter().all(|i| { @@ -1175,15 +1132,13 @@ async fn test_update_fungible_token_static_details_job() { String::from("EPjFWdd5AufqSSqeM2qN1xzybapC8G4wEGGkZwyTDt1v"), ]; let sut = Scheduler::new(setup.rocks_db.clone(), Some(well_known_fungible_pks)); - Scheduler::run_in_background(sut).await; + Scheduler::run_in_background(sut, CancellationToken::new()).await; - // sync data to Postgre - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup.synchronizer.full_syncronize(&shutdown_rx, Fungible).await.unwrap(); - setup.synchronizer.full_syncronize(&shutdown_rx, NonFungible).await.unwrap(); + setup.synchronizer.full_syncronize(CancellationToken::new(), Fungible).await.unwrap(); + setup.synchronizer.full_syncronize(CancellationToken::new(), NonFungible).await.unwrap(); let request_2 = serde_json::from_str(request_str).unwrap(); - let response_2 = setup.das_api.get_asset_batch(request_2, mutexed_tasks.clone()).await.unwrap(); + let response_2 = setup.das_api.get_asset_batch(request_2).await.unwrap(); assert_eq!(response_2.as_array().unwrap().len(), 2); response_2.as_array().unwrap().iter().all(|i| { diff --git a/integration_tests/src/synchronizer_tests.rs b/integration_tests/src/synchronizer_tests.rs index 29a3d1951..520a684b9 100644 --- a/integration_tests/src/synchronizer_tests.rs +++ b/integration_tests/src/synchronizer_tests.rs @@ -1,23 +1,17 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use entities::{ - api_req_params::{ - GetAsset, GetAssetBatch, GetAssetsByAuthority, GetAssetsByGroup, GetAssetsByOwner, - SearchAssets, - }, + api_req_params::{GetAssetsByAuthority, GetAssetsByGroup, GetAssetsByOwner}, enums::AssetType, }; use function_name::named; use itertools::Itertools; use serial_test::serial; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; +use tokio_util::sync::CancellationToken; use super::common::*; -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_full_sync_core_get_assets_by_authority() { @@ -41,11 +35,9 @@ async fn test_full_sync_core_get_assets_by_authority() { single_db_index_seed_events(&setup, seeds.iter().collect_vec()).await; - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup .synchronizer - .full_syncronize(&shutdown_rx.resubscribe(), AssetType::NonFungible) + .full_syncronize(CancellationToken::new(), AssetType::NonFungible) .await .unwrap(); @@ -61,15 +53,12 @@ async fn test_full_sync_core_get_assets_by_authority() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByAuthority = serde_json::from_str(request).unwrap(); - let response = - setup.das_api.get_assets_by_authority(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_authority(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_full_sync_core_get_assets_by_group() { @@ -93,11 +82,9 @@ async fn test_full_sync_core_get_assets_by_group() { single_db_index_seed_events(&setup, seeds.iter().collect_vec()).await; - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup .synchronizer - .full_syncronize(&shutdown_rx.resubscribe(), AssetType::NonFungible) + .full_syncronize(CancellationToken::new(), AssetType::NonFungible) .await .unwrap(); @@ -114,14 +101,12 @@ async fn test_full_sync_core_get_assets_by_group() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByGroup = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_group(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_group(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_full_sync_core_get_assets_by_owner() { @@ -143,11 +128,9 @@ async fn test_full_sync_core_get_assets_by_owner() { single_db_index_seed_events(&setup, seeds.iter().collect_vec()).await; - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup .synchronizer - .full_syncronize(&shutdown_rx.resubscribe(), AssetType::NonFungible) + .full_syncronize(CancellationToken::new(), AssetType::NonFungible) .await .unwrap(); @@ -163,14 +146,12 @@ async fn test_full_sync_core_get_assets_by_owner() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByOwner = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_owner(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_owner(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_full_sync_core_and_regular_nfts_get_assets_by_owner() { @@ -197,11 +178,9 @@ async fn test_full_sync_core_and_regular_nfts_get_assets_by_owner() { single_db_index_seed_events(&setup, seeds.iter().collect_vec()).await; - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - setup .synchronizer - .full_syncronize(&shutdown_rx.resubscribe(), AssetType::NonFungible) + .full_syncronize(CancellationToken::new(), AssetType::NonFungible) .await .unwrap(); @@ -217,9 +196,7 @@ async fn test_full_sync_core_and_regular_nfts_get_assets_by_owner() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAssetsByOwner = serde_json::from_str(request).unwrap(); - let response = setup.das_api.get_assets_by_owner(request, mutexed_tasks.clone()).await.unwrap(); + let response = setup.das_api.get_assets_by_owner(request).await.unwrap(); insta::assert_json_snapshot!(name, response); } diff --git a/integration_tests/src/token_tests.rs b/integration_tests/src/token_tests.rs index f124cf3c7..af183364e 100644 --- a/integration_tests/src/token_tests.rs +++ b/integration_tests/src/token_tests.rs @@ -1,24 +1,15 @@ -use std::{collections::HashMap, sync::Arc}; - -use entities::{api_req_params::GetAsset, enums::AssetType}; +use entities::api_req_params::GetAsset; use function_name::named; use itertools::Itertools; -use nft_ingester::api::dapi::{ - response::{AssetList, TokenAccountsList}, - rpc_asset_models::Asset, -}; +use nft_ingester::api::dapi::rpc_asset_models::Asset; use serial_test::serial; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; use crate::common::{ index_seed_events, seed_token_mints, trim_test_name, well_known_fungible_tokens, Network, SeedEvent, TestSetup, TestSetupOptions, }; -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[serial] #[named] async fn test_fungible_token_mint_freeze_authority() { @@ -44,10 +35,8 @@ async fn test_fungible_token_mint_freeze_authority() { } "#; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let request: GetAsset = serde_json::from_str(request).unwrap(); - let response_value = setup.das_api.get_asset(request, mutexed_tasks.clone()).await.unwrap(); + let response_value = setup.das_api.get_asset(request).await.unwrap(); let asset: Asset = serde_json::from_value::(response_value.clone()).expect("Cannot parse 'Asset'."); diff --git a/integrity_verification/src/diff_checker.rs b/integrity_verification/src/diff_checker.rs index a3f7d34ae..932670fbe 100644 --- a/integrity_verification/src/diff_checker.rs +++ b/integrity_verification/src/diff_checker.rs @@ -9,7 +9,7 @@ use serde_json::{json, Value}; use solana_client::nonblocking::rpc_client::RpcClient; use solana_program::pubkey::Pubkey; use solana_sdk::commitment_config::CommitmentLevel; -use tokio::sync::broadcast::Receiver; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use usecase::{ bigtable::BigTableClient, proofs::MaybeProofChecker, slots_collector::SlotsCollector, @@ -214,8 +214,13 @@ where if let Some(diff) = diff_with_responses.diff { test_failed = true; error!("{}: mismatch responses: req: {:#?}, diff: {}", req.method, req, diff); - let (_tx, rx) = tokio::sync::broadcast::channel::<()>(1); - self.try_collect_slots(req, &diff_with_responses.reference_response, &rx).await; + // this will never be cancelled + self.try_collect_slots( + req, + &diff_with_responses.reference_response, + CancellationToken::new(), + ) + .await; } if req.method == GET_ASSET_PROOF_METHOD { @@ -404,7 +409,12 @@ impl DiffChecker where T: IntegrityVerificationKeysFetcher + Send + Sync, { - async fn try_collect_slots(&self, req: &Body, reference_response: &Value, rx: &Receiver<()>) { + async fn try_collect_slots( + &self, + req: &Body, + reference_response: &Value, + cancellation_token: CancellationToken, + ) { let collect_tools = match &self.collect_slots_tools { None => return, Some(collect_tools) => collect_tools, @@ -435,7 +445,7 @@ where }, }; if let Ok(tree_id) = Pubkey::from_str(tree_id) { - collect_tools.collect_slots(asset_id, tree_id, slot, rx).await + collect_tools.collect_slots(asset_id, tree_id, slot, cancellation_token).await } } @@ -481,7 +491,13 @@ where } impl CollectSlotsTools { - async fn collect_slots(&self, asset: &str, tree_key: Pubkey, slot: u64, rx: &Receiver<()>) { + async fn collect_slots( + &self, + asset: &str, + tree_key: Pubkey, + slot: u64, + cancellation_token: CancellationToken, + ) { let slots_collector = SlotsCollector::new( Arc::new(FileSlotsDumper::new(self.format_filename(&tree_key.to_string(), asset))), self.bigtable_client.big_table_inner_client.clone(), @@ -489,7 +505,7 @@ impl CollectSlotsTools { ); info!("Start collecting slots for {}", tree_key); - slots_collector.collect_slots(&tree_key, slot, 0, rx).await; + slots_collector.collect_slots(&tree_key, slot, 0, cancellation_token).await; info!("Collected slots for {}", tree_key); } @@ -544,8 +560,8 @@ mod tests { use std::str::FromStr; use solana_program::pubkey::Pubkey; + use tokio_util::sync::CancellationToken; - let (_tx, rx) = tokio::sync::broadcast::channel::<()>(1); create_test_diff_checker() .await .collect_slots_tools @@ -554,7 +570,7 @@ mod tests { "BAtEs7TuGm2hP2owc9cTit2TNfVzpPFyQAAvkDWs6tDm", Pubkey::from_str("4FZcSBJkhPeNAkXecmKnnqHy93ABWzi3Q5u9eXkUfxVE").unwrap(), 244259062, - &rx, + CancellationToken::new(), ) .await; } diff --git a/integrity_verification/src/main.rs b/integrity_verification/src/main.rs index 9e0817ca7..b314a2405 100644 --- a/integrity_verification/src/main.rs +++ b/integrity_verification/src/main.rs @@ -48,7 +48,7 @@ async fn main() { start_metrics(metrics.registry, Some(config.metrics_port)).await; let mut tasks = JoinSet::new(); - let cancel_token = CancellationToken::new(); + let cancellation_token = CancellationToken::new(); match config.test_source_mode { TestSourceMode::File => { let diff_checker = DiffChecker::new( @@ -73,7 +73,7 @@ async fn main() { config.run_assets_tests, diff_checker, metrics.integrity_verification_metrics.clone(), - cancel_token.clone(), + cancellation_token.clone(), ) .await; }, @@ -107,15 +107,13 @@ async fn main() { config.run_assets_tests, diff_checker, metrics.integrity_verification_metrics.clone(), - cancel_token.clone(), + cancellation_token.clone(), ) .await; }, }; - usecase::graceful_stop::listen_shutdown().await; - cancel_token.cancel(); - usecase::graceful_stop::graceful_stop(&mut tasks).await; + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; } macro_rules! spawn_test { diff --git a/interface/Cargo.toml b/interface/Cargo.toml index ef755c516..b8363e805 100644 --- a/interface/Cargo.toml +++ b/interface/Cargo.toml @@ -22,5 +22,6 @@ anchor-lang = { workspace = true } serde_json = { workspace = true } serde_derive = { workspace = true } tokio = { workspace = true } +tokio-util = { workspace = true } jsonrpc-core = { workspace = true } bubblegum-batch-sdk = { workspace = true } diff --git a/interface/src/fork_cleaner.rs b/interface/src/fork_cleaner.rs index ccd3872de..d52425802 100644 --- a/interface/src/fork_cleaner.rs +++ b/interface/src/fork_cleaner.rs @@ -3,7 +3,7 @@ use std::collections::HashSet; use async_trait::async_trait; use entities::models::{ClItem, ForkedItem, LeafSignatureAllData}; use solana_sdk::{pubkey::Pubkey, signature::Signature}; -use tokio::sync::broadcast::Receiver; +use tokio_util::sync::CancellationToken; #[async_trait] pub trait CompressedTreeChangesManager { @@ -16,6 +16,6 @@ pub trait CompressedTreeChangesManager { #[async_trait] pub trait ForkChecker { - fn get_all_non_forked_slots(&self, rx: Receiver<()>) -> HashSet; + fn get_all_non_forked_slots(&self, cancellation_token: CancellationToken) -> HashSet; fn last_slot_for_check(&self) -> u64; } diff --git a/nft_ingester/benches/ingester_benchmark.rs b/nft_ingester/benches/ingester_benchmark.rs index fbe704f90..896ae4db9 100644 --- a/nft_ingester/benches/ingester_benchmark.rs +++ b/nft_ingester/benches/ingester_benchmark.rs @@ -70,7 +70,6 @@ fn ingest_benchmark(c: &mut Criterion) { let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let transactions_storage = Storage::open( &format!("{}{}", tx_storage_dir.path().to_str().unwrap(), "/test_rocks"), - mutexed_tasks.clone(), red_metrics, MigrationState::Last, ) diff --git a/nft_ingester/benches/integrated_benchmark.rs b/nft_ingester/benches/integrated_benchmark.rs index 0f196308d..3137e72dd 100644 --- a/nft_ingester/benches/integrated_benchmark.rs +++ b/nft_ingester/benches/integrated_benchmark.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use criterion::{criterion_group, criterion_main, Criterion}; use entities::api_req_params::SearchAssets; use metrics_utils::ApiMetricsConfig; -use nft_ingester::{api::middleware::JsonDownloaderMiddleware, index_syncronizer::Synchronizer}; +use nft_ingester::{api::middleware::JsonDownloaderMiddleware, index_synchronizer::Synchronizer}; use rocks_db::storage_traits::AssetIndexReader; use setup::TestEnvironment; use testcontainers::clients::Cli; diff --git a/nft_ingester/benches/synchronizer_benchmark.rs b/nft_ingester/benches/synchronizer_benchmark.rs index 6bdef5642..e0d677518 100644 --- a/nft_ingester/benches/synchronizer_benchmark.rs +++ b/nft_ingester/benches/synchronizer_benchmark.rs @@ -21,7 +21,7 @@ async fn bench_synchronize(env: Arc>, batch_size: usize) { .await .unwrap(); let metrics = Arc::new(SynchronizerMetricsConfig::new()); - let syncronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), env.pg_env.client.clone(), @@ -33,14 +33,14 @@ async fn bench_synchronize(env: Arc>, batch_size: usize) { ); let (_, rx) = tokio::sync::broadcast::channel::<()>(1); - syncronizer.synchronize_asset_indexes(&rx, 0).await.unwrap(); + synchronizer.synchronize_asset_indexes(&rx, 0).await.unwrap(); } fn sync_benchmark(c: &mut Criterion) { let cli: Cli = Cli::default(); let rt = tokio::runtime::Runtime::new().unwrap(); let (env, _generated_assets) = rt.block_on(setup_environment(&cli)); - let mut group = c.benchmark_group("Syncronizer Group"); + let mut group = c.benchmark_group("Synchronizer Group"); group.sample_size(10); group.measurement_time(std::time::Duration::from_secs(60)); let env = Arc::new(env); diff --git a/nft_ingester/src/ack.rs b/nft_ingester/src/ack.rs index e2884c0f8..c3da4ddc0 100644 --- a/nft_ingester/src/ack.rs +++ b/nft_ingester/src/ack.rs @@ -1,29 +1,24 @@ -use std::{collections::HashMap, sync::Arc}; +use std::collections::HashMap; use plerkle_messenger::{redis_messenger::RedisMessenger, Messenger, MessengerConfig}; use tokio::{ - sync::{ - broadcast::Receiver, - mpsc::{unbounded_channel, UnboundedSender}, - Mutex, - }, - task::{JoinError, JoinSet}, + sync::mpsc::{unbounded_channel, UnboundedSender}, time::{interval, Duration}, }; +use tokio_util::sync::CancellationToken; use tracing::log::error; pub async fn create_ack_channel( - shutdown_rx: Receiver<()>, config: MessengerConfig, - tasks: Arc>>>, + cancellation_token: CancellationToken, ) -> UnboundedSender<(&'static str, String)> { let (tx, mut rx) = unbounded_channel::<(&'static str, String)>(); - tasks.lock().await.spawn(async move { + usecase::executor::spawn(async move { let mut interval = interval(Duration::from_millis(100)); let mut acks: HashMap<&str, Vec> = HashMap::new(); let source = RedisMessenger::new(config).await; if let Ok(mut msg) = source { - while shutdown_rx.is_empty() { + while !cancellation_token.is_cancelled() { tokio::select! { _ = interval.tick() => { if acks.is_empty() { @@ -45,7 +40,6 @@ pub async fn create_ack_channel( } } } - Ok(()) }); tx diff --git a/nft_ingester/src/api/api_impl.rs b/nft_ingester/src/api/api_impl.rs index 475252fba..c72c69d28 100644 --- a/nft_ingester/src/api/api_impl.rs +++ b/nft_ingester/src/api/api_impl.rs @@ -26,10 +26,6 @@ use postgre_client::PgClient; use rocks_db::Storage; use serde_json::{json, Value}; use solana_sdk::pubkey::Pubkey; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use usecase::validation::{validate_opt_pubkey, validate_pubkey}; use self::util::ApiRequest; @@ -304,11 +300,7 @@ where Ok(json!(res?)) } - pub async fn get_asset( - &self, - payload: GetAsset, - tasks: Arc>>>, - ) -> Result { + pub async fn get_asset(&self, payload: GetAsset) -> Result { let label = "get_asset"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); @@ -323,7 +315,6 @@ where self.json_downloader.clone(), self.json_persister.clone(), self.json_middleware_config.max_urls_to_parse, - tasks, self.storage_service_base_path.clone(), self.token_price_fetcher.clone(), self.metrics.clone(), @@ -340,11 +331,7 @@ where Ok(json!(res)) } - pub async fn get_asset_batch( - &self, - payload: GetAssetBatch, - tasks: Arc>>>, - ) -> Result { + pub async fn get_asset_batch(&self, payload: GetAssetBatch) -> Result { let label = "get_asset_batch"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); @@ -367,7 +354,6 @@ where self.json_downloader.clone(), self.json_persister.clone(), self.json_middleware_config.max_urls_to_parse, - tasks, self.storage_service_base_path.clone(), self.token_price_fetcher.clone(), self.metrics.clone(), @@ -383,15 +369,13 @@ where pub async fn get_assets_by_owner( &self, payload: GetAssetsByOwner, - tasks: Arc>>>, ) -> Result { let label = "get_assets_by_owner"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); - let res = self - .process_request(self.pg_client.clone(), self.rocks_db.clone(), payload, tasks) - .await?; + let res = + self.process_request(self.pg_client.clone(), self.rocks_db.clone(), payload).await?; self.metrics.set_latency(label, latency_timer.elapsed().as_millis() as f64); @@ -401,15 +385,13 @@ where pub async fn get_assets_by_group( &self, payload: GetAssetsByGroup, - tasks: Arc>>>, ) -> Result { let label = "get_assets_by_group"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); - let res = self - .process_request(self.pg_client.clone(), self.rocks_db.clone(), payload, tasks) - .await?; + let res = + self.process_request(self.pg_client.clone(), self.rocks_db.clone(), payload).await?; self.metrics.set_latency(label, latency_timer.elapsed().as_millis() as f64); @@ -419,15 +401,13 @@ where pub async fn get_assets_by_creator( &self, payload: GetAssetsByCreator, - tasks: Arc>>>, ) -> Result { let label = "get_assets_by_creator"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); - let res = self - .process_request(self.pg_client.clone(), self.rocks_db.clone(), payload, tasks) - .await?; + let res = + self.process_request(self.pg_client.clone(), self.rocks_db.clone(), payload).await?; self.metrics.set_latency(label, latency_timer.elapsed().as_millis() as f64); @@ -437,15 +417,13 @@ where pub async fn get_assets_by_authority( &self, payload: GetAssetsByAuthority, - tasks: Arc>>>, ) -> Result { let label = "get_assets_by_authority"; self.metrics.inc_requests(label); let latency_timer = Instant::now(); - let res = self - .process_request(self.pg_client.clone(), self.rocks_db.clone(), payload, tasks) - .await?; + let res = + self.process_request(self.pg_client.clone(), self.rocks_db.clone(), payload).await?; self.metrics.set_latency(label, latency_timer.elapsed().as_millis() as f64); @@ -518,19 +496,14 @@ where Ok(json!(res)) } - pub async fn search_assets( - &self, - payload: SearchAssets, - tasks: Arc>>>, - ) -> Result { + pub async fn search_assets(&self, payload: SearchAssets) -> Result { // use names of the filter fields as a label for better understanding of the endpoint usage let label = payload.extract_some_fields(); self.metrics.inc_search_asset_requests(&label); let latency_timer = Instant::now(); - let res = self - .process_request(self.pg_client.clone(), self.rocks_db.clone(), payload, tasks) - .await?; + let res = + self.process_request(self.pg_client.clone(), self.rocks_db.clone(), payload).await?; self.metrics.set_search_asset_latency(&label, latency_timer.elapsed().as_millis() as f64); self.metrics.set_latency("search_asset", latency_timer.elapsed().as_millis() as f64); @@ -623,7 +596,6 @@ where pg_client: Arc, rocks_db: Arc, payload: T, - tasks: Arc>>>, ) -> Result where T: TryInto, @@ -654,7 +626,6 @@ where self.json_downloader.clone(), self.json_persister.clone(), self.json_middleware_config.max_urls_to_parse, - tasks, self.account_balance_getter.clone(), self.storage_service_base_path.clone(), self.token_price_fetcher.clone(), diff --git a/nft_ingester/src/api/backfilling_state_consistency.rs b/nft_ingester/src/api/backfilling_state_consistency.rs index 4bceac765..151238e32 100644 --- a/nft_ingester/src/api/backfilling_state_consistency.rs +++ b/nft_ingester/src/api/backfilling_state_consistency.rs @@ -10,11 +10,7 @@ use entities::enums::{AssetType, ASSET_TYPES}; use interface::consistency_check::ConsistencyChecker; use jsonrpc_core::Call; use rocks_db::Storage; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; -use tracing::info; +use tokio_util::sync::CancellationToken; use crate::api::synchronization_state_consistency::CATCH_UP_SEQUENCES_TIMEOUT_SEC; @@ -33,36 +29,33 @@ impl BackfillingStateConsistencyChecker { pub(crate) async fn run( &self, - tasks: Arc>>>, - rx: tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, rocks_db: Arc, _consistence_backfilling_slots_threshold: u64, ) { for asset_type in ASSET_TYPES { let _rocks_db = rocks_db.clone(); - let mut rx = rx.resubscribe(); let _overwhelm_backfill_gap = match asset_type { AssetType::NonFungible => self.overwhelm_nft_backfill_gap.clone(), AssetType::Fungible => self.overwhelm_fungible_backfill_gap.clone(), }; - tasks.lock().await.spawn(async move { - while rx.is_empty() { - // TODO: refactor this to use parameter from storage and last slot from slot storage - // overwhelm_backfill_gap.store( - // rocks_db.bubblegum_slots.iter_start().count().saturating_add(rocks_db.ingestable_slots.iter_start().count()) - // >= consistence_backfilling_slots_threshold as usize, - // Ordering::Relaxed, - // ); - tokio::select! { - _ = tokio::time::sleep(Duration::from_secs(CATCH_UP_SEQUENCES_TIMEOUT_SEC)) => {}, - _ = rx.recv() => { - info!("Received stop signal, stopping BackfillingStateConsistencyChecker..."); - return Ok(()); - } + let cancellation_token = cancellation_token.child_token(); + usecase::executor::spawn(async move { + while !cancellation_token.is_cancelled() { + // TODO: refactor this to use parameter from storage and last slot from slot storage + // overwhelm_backfill_gap.store( + // rocks_db.bubblegum_slots.iter_start().count().saturating_add(rocks_db.ingestable_slots.iter_start().count()) + // >= consistence_backfilling_slots_threshold as usize, + // Ordering::Relaxed, + // ); + // + cancellation_token + .run_until_cancelled(tokio::time::sleep(Duration::from_secs( + CATCH_UP_SEQUENCES_TIMEOUT_SEC, + ))) + .await; } - } - Ok(()) - }); + }); } } } diff --git a/nft_ingester/src/api/builder.rs b/nft_ingester/src/api/builder.rs index 9b8509eeb..1333d6598 100644 --- a/nft_ingester/src/api/builder.rs +++ b/nft_ingester/src/api/builder.rs @@ -8,10 +8,6 @@ use entities::api_req_params::{ use interface::consistency_check::ConsistencyChecker; use jsonrpc_core::{types::params::Params, MetaIoHandler}; use rocks_db::Storage; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use usecase::proofs::MaybeProofChecker; use crate::{ @@ -36,7 +32,6 @@ impl RpcApiBuilder { Storage, >, consistency_checkers: Vec>, - tasks: Arc>>>, ) -> Result, DasApiError> { let mut module = MetaIoHandler::::new( Default::default(), @@ -59,15 +54,13 @@ impl RpcApiBuilder { module.add_alias("getAssetProof", "get_asset_proof"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_asset", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => api.get_asset(payload, tasks).await.map_err(Into::into), + Ok(payload) => api.get_asset(payload).await.map_err(Into::into), Err(_) => api - .get_asset(rpc_params.parse::()?.into(), tasks) + .get_asset(rpc_params.parse::()?.into()) .await .map_err(Into::into), } @@ -76,20 +69,13 @@ impl RpcApiBuilder { module.add_alias("getAsset", "get_asset"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_assets_by_owner", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => { - api.get_assets_by_owner(payload, tasks).await.map_err(Into::into) - }, + Ok(payload) => api.get_assets_by_owner(payload).await.map_err(Into::into), Err(_) => api - .get_assets_by_owner( - rpc_params.parse::()?.into(), - tasks, - ) + .get_assets_by_owner(rpc_params.parse::()?.into()) .await .map_err(Into::into), } @@ -98,20 +84,13 @@ impl RpcApiBuilder { module.add_alias("getAssetsByOwner", "get_assets_by_owner"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_assets_by_creator", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => { - api.get_assets_by_creator(payload, tasks).await.map_err(Into::into) - }, + Ok(payload) => api.get_assets_by_creator(payload).await.map_err(Into::into), Err(_) => api - .get_assets_by_creator( - rpc_params.parse::()?.into(), - tasks, - ) + .get_assets_by_creator(rpc_params.parse::()?.into()) .await .map_err(Into::into), } @@ -120,19 +99,14 @@ impl RpcApiBuilder { module.add_alias("getAssetsByCreator", "get_assets_by_creator"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_assets_by_authority", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => { - api.get_assets_by_authority(payload, tasks).await.map_err(Into::into) - }, + Ok(payload) => api.get_assets_by_authority(payload).await.map_err(Into::into), Err(_) => api .get_assets_by_authority( rpc_params.parse::()?.into(), - tasks, ) .await .map_err(Into::into), @@ -142,20 +116,13 @@ impl RpcApiBuilder { module.add_alias("getAssetsByAuthority", "get_assets_by_authority"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_assets_by_group", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => { - api.get_assets_by_group(payload, tasks).await.map_err(Into::into) - }, + Ok(payload) => api.get_assets_by_group(payload).await.map_err(Into::into), Err(_) => api - .get_assets_by_group( - rpc_params.parse::()?.into(), - tasks, - ) + .get_assets_by_group(rpc_params.parse::()?.into()) .await .map_err(Into::into), } @@ -164,15 +131,13 @@ impl RpcApiBuilder { module.add_alias("getAssetsByGroup", "get_assets_by_group"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("get_asset_batch", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => api.get_asset_batch(payload, tasks).await.map_err(Into::into), + Ok(payload) => api.get_asset_batch(payload).await.map_err(Into::into), Err(_) => api - .get_asset_batch(rpc_params.parse::()?.into(), tasks) + .get_asset_batch(rpc_params.parse::()?.into()) .await .map_err(Into::into), } @@ -199,15 +164,13 @@ impl RpcApiBuilder { module.add_alias("getGrouping", "get_grouping"); let cloned_api = api.clone(); - let cloned_tasks = tasks.clone(); module.add_method("search_assets", move |rpc_params: Params| { let api = cloned_api.clone(); - let tasks = cloned_tasks.clone(); async move { match rpc_params.clone().parse::() { - Ok(payload) => api.search_assets(payload, tasks).await.map_err(Into::into), + Ok(payload) => api.search_assets(payload).await.map_err(Into::into), Err(_) => api - .search_assets(rpc_params.parse::()?.into(), tasks) + .search_assets(rpc_params.parse::()?.into()) .await .map_err(Into::into), } diff --git a/nft_ingester/src/api/dapi/asset.rs b/nft_ingester/src/api/dapi/asset.rs index ff2331088..680412da6 100644 --- a/nft_ingester/src/api/dapi/asset.rs +++ b/nft_ingester/src/api/dapi/asset.rs @@ -23,10 +23,6 @@ use rocks_db::{ Storage, }; use solana_sdk::pubkey::Pubkey; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use tracing::error; use crate::api::dapi::rpc_asset_models::FullAsset; @@ -169,7 +165,6 @@ pub async fn get_by_ids< json_downloader: Option>, json_persister: Option>, max_json_to_download: usize, - tasks: Arc>>>, // We need owner_address if we want to query fungible token accounts owner_address: &Option, token_price_fetcher: Arc, @@ -315,11 +310,10 @@ pub async fn get_by_ids< if let Some(json_persister) = json_persister { if !download_results.is_empty() { - tasks.lock().await.spawn(async move { + usecase::executor::spawn(async move { if let Err(e) = json_persister.persist_response(download_results).await { error!("Could not persist downloaded JSONs: {:?}", e); } - Ok(()) }); } } diff --git a/nft_ingester/src/api/dapi/change_logs.rs b/nft_ingester/src/api/dapi/change_logs.rs index 089f2b532..c2a2cbb1c 100644 --- a/nft_ingester/src/api/dapi/change_logs.rs +++ b/nft_ingester/src/api/dapi/change_logs.rs @@ -275,7 +275,7 @@ fn get_asset_proof( let metrics = metrics.clone(); let cloned_checker = proof_checker.clone(); let asset_id = *asset_id; - tokio::spawn(async move { + usecase::executor::spawn(async move { match cloned_checker .check_proof(tree_id, initial_proofs, leaf_data.nonce as u32, lf.to_bytes()) .await diff --git a/nft_ingester/src/api/dapi/get_asset.rs b/nft_ingester/src/api/dapi/get_asset.rs index 9dc5d0a34..45a9fc093 100644 --- a/nft_ingester/src/api/dapi/get_asset.rs +++ b/nft_ingester/src/api/dapi/get_asset.rs @@ -9,10 +9,6 @@ use interface::{ use metrics_utils::ApiMetricsConfig; use rocks_db::{errors::StorageError, Storage}; use solana_sdk::pubkey::Pubkey; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use super::asset_preview::populate_previews_slice; use crate::api::dapi::{asset, rpc_asset_convertors::asset_to_rpc, rpc_asset_models::Asset}; @@ -30,7 +26,6 @@ pub async fn get_asset< json_downloader: Option>, json_persister: Option>, max_json_to_download: usize, - tasks: Arc>>>, storage_service_base_path: Option, token_price_fetcher: Arc, metrics: Arc, @@ -43,7 +38,6 @@ pub async fn get_asset< json_downloader, json_persister, max_json_to_download, - tasks, &None, token_price_fetcher, metrics, diff --git a/nft_ingester/src/api/dapi/get_asset_batch.rs b/nft_ingester/src/api/dapi/get_asset_batch.rs index 42c893d10..0456749e0 100644 --- a/nft_ingester/src/api/dapi/get_asset_batch.rs +++ b/nft_ingester/src/api/dapi/get_asset_batch.rs @@ -9,10 +9,6 @@ use interface::{ use metrics_utils::ApiMetricsConfig; use rocks_db::{errors::StorageError, Storage}; use solana_sdk::pubkey::Pubkey; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use super::asset_preview::populate_previews_opt; use crate::api::dapi::{asset, rpc_asset_convertors::asset_to_rpc, rpc_asset_models::Asset}; @@ -30,7 +26,6 @@ pub async fn get_asset_batch< json_downloader: Option>, json_persister: Option>, max_json_to_download: usize, - tasks: Arc>>>, storage_service_base_path: Option, token_price_fetcher: Arc, metrics: Arc, @@ -43,7 +38,6 @@ pub async fn get_asset_batch< json_downloader, json_persister, max_json_to_download, - tasks, &None, token_price_fetcher, metrics, diff --git a/nft_ingester/src/api/dapi/search_assets.rs b/nft_ingester/src/api/dapi/search_assets.rs index 1857a9a9c..c44ae3e1c 100644 --- a/nft_ingester/src/api/dapi/search_assets.rs +++ b/nft_ingester/src/api/dapi/search_assets.rs @@ -13,10 +13,6 @@ use interface::{ use metrics_utils::ApiMetricsConfig; use rocks_db::{errors::StorageError, Storage}; use solana_sdk::pubkey::Pubkey; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; use tracing::error; use super::asset_preview::populate_previews; @@ -47,7 +43,6 @@ pub async fn search_assets< json_downloader: Option>, json_persister: Option>, max_json_to_download: usize, - tasks: Arc>>>, account_balance_getter: Arc, storage_service_base_path: Option, token_price_fetcher: Arc, @@ -74,7 +69,6 @@ pub async fn search_assets< json_downloader, json_persister, max_json_to_download, - tasks, token_price_fetcher.clone(), metrics, tree_gaps_checker, @@ -121,7 +115,6 @@ async fn fetch_assets< json_downloader: Option>, json_persister: Option>, max_json_to_download: usize, - tasks: Arc>>>, token_price_fetcher: Arc, metrics: Arc, tree_gaps_checker: &Option>, @@ -181,7 +174,6 @@ async fn fetch_assets< json_downloader, json_persister, max_json_to_download, - tasks, &owner_address, token_price_fetcher, metrics, diff --git a/nft_ingester/src/api/service.rs b/nft_ingester/src/api/service.rs index 229b56f39..61db9c648 100644 --- a/nft_ingester/src/api/service.rs +++ b/nft_ingester/src/api/service.rs @@ -12,12 +12,8 @@ use metrics_utils::ApiMetricsConfig; use multer::Multipart; use postgre_client::PgClient; use rocks_db::Storage; -use tokio::{ - fs::File, - io::AsyncWriteExt, - sync::{broadcast::Receiver, Mutex}, - task::{JoinError, JoinSet}, -}; +use tokio::{fs::File, io::AsyncWriteExt}; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use usecase::proofs::MaybeProofChecker; use uuid::Uuid; @@ -54,7 +50,7 @@ pub(crate) struct MiddlewaresData { pub async fn start_api( pg_client: Arc, rocks_db: Arc, - rx: Receiver<()>, + cancellation_token: CancellationToken, metrics: Arc, port: u16, proof_checker: Option>, @@ -63,7 +59,6 @@ pub async fn start_api( json_downloader: Option>, json_persister: Option>, json_middleware_config: Option, - tasks: Arc>>>, archives_dir: &str, consistence_synchronization_api_threshold: Option, consistence_backfilling_slots_threshold: Option, @@ -86,8 +81,7 @@ pub async fn start_api( Arc::new(SynchronizationStateConsistencyChecker::new()); synchronization_state_consistency_checker .run( - tasks.clone(), - rx.resubscribe(), + cancellation_token.child_token(), pg_client.clone(), rocks_db.clone(), consistence_synchronization_api_threshold, @@ -101,8 +95,7 @@ pub async fn start_api( Arc::new(BackfillingStateConsistencyChecker::new()); backfilling_state_consistency_checker .run( - tasks.clone(), - rx.resubscribe(), + cancellation_token.child_token(), rocks_db.clone(), consistence_backfilling_slots_threshold, ) @@ -132,11 +125,10 @@ pub async fn start_api( api, Some(MiddlewaresData { response_middleware, request_middleware, consistency_checkers }), addr, - tasks, batch_mint_service_port, file_storage_path, pg_client, - rx, + cancellation_token.child_token(), ) .await } @@ -153,16 +145,14 @@ async fn run_api( >, middlewares_data: Option, addr: SocketAddr, - tasks: Arc>>>, batch_mint_service_port: Option, file_storage_path: &str, pg_client: Arc, - shutdown_rx: Receiver<()>, + cancellation_token: CancellationToken, ) -> Result<(), DasApiError> { let rpc = RpcApiBuilder::build( api, middlewares_data.clone().map(|m| m.consistency_checkers).unwrap_or_default(), - tasks, )?; let runtime = tokio::runtime::Builder::new_multi_thread() .worker_threads(RUNTIME_WORKER_THREAD_COUNT) @@ -187,7 +177,7 @@ async fn run_api( let server = builder.start_http(&addr); if let Some(port) = batch_mint_service_port { run_batch_mint_service( - shutdown_rx.resubscribe(), + cancellation_token.child_token(), port, file_storage_path.to_string(), pg_client, @@ -199,7 +189,7 @@ async fn run_api( info!("API Server Started {}", server.address().to_string()); loop { - if !shutdown_rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Shutting down server"); runtime.shutdown_background(); break; @@ -304,7 +294,7 @@ impl BatchMintService { } async fn run_batch_mint_service( - mut shutdown_rx: Receiver<()>, + cancellation_token: CancellationToken, port: u16, file_storage_path: String, pg_client: Arc, @@ -320,7 +310,7 @@ async fn run_batch_mint_service( } }); let server = Server::bind(&addr).serve(make_svc).with_graceful_shutdown(async { - shutdown_rx.recv().await.unwrap(); + cancellation_token.cancelled().await; }); if let Err(e) = server.await { error!("server error: {}", e); diff --git a/nft_ingester/src/api/synchronization_state_consistency.rs b/nft_ingester/src/api/synchronization_state_consistency.rs index a33968e83..f4021ce45 100644 --- a/nft_ingester/src/api/synchronization_state_consistency.rs +++ b/nft_ingester/src/api/synchronization_state_consistency.rs @@ -13,11 +13,7 @@ use postgre_client::{storage_traits::AssetIndexStorage, PgClient}; use rocks_db::{ key_encoders::decode_u64x2_pubkey, storage_traits::AssetUpdateIndexStorage, Storage, }; -use tokio::{ - sync::Mutex, - task::{JoinError, JoinSet}, -}; -use tracing::info; +use tokio_util::sync::CancellationToken; pub(crate) const CATCH_UP_SEQUENCES_TIMEOUT_SEC: u64 = 30; const INDEX_STORAGE_DEPENDS_METHODS: &[&str] = &[ @@ -50,8 +46,7 @@ impl SynchronizationStateConsistencyChecker { pub(crate) async fn run( &self, - tasks: Arc>>>, - rx: tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, pg_client: Arc, rocks_db: Arc, synchronization_api_threshold: u64, @@ -63,10 +58,11 @@ impl SynchronizationStateConsistencyChecker { }; let pg_client = pg_client.clone(); let rocks_db = rocks_db.clone(); - let mut rx = rx.resubscribe(); - tasks.lock().await.spawn(async move { - while rx.is_empty() { - let Ok(Some(index_seq)) = pg_client.fetch_last_synced_id(asset_type).await else { + let cancellation_token = cancellation_token.child_token(); + usecase::executor::spawn(async move { + while !cancellation_token.is_cancelled() { + let Ok(Some(index_seq)) = pg_client.fetch_last_synced_id(asset_type).await + else { continue; }; let Ok(decoded_index_update_key) = decode_u64x2_pubkey(index_seq) else { @@ -82,21 +78,18 @@ impl SynchronizationStateConsistencyChecker { }; overwhelm_seq_gap.store( - primary_update_key - .seq - .saturating_sub(decoded_index_update_key.seq) + primary_update_key.seq.saturating_sub(decoded_index_update_key.seq) >= synchronization_api_threshold, Ordering::Relaxed, ); + tokio::select! { _ = tokio::time::sleep(Duration::from_secs(CATCH_UP_SEQUENCES_TIMEOUT_SEC))=> {}, - _ = rx.recv() => { - info!("Received stop signal, stopping SynchronizationStateConsistencyChecker..."); - return Ok(()); + _ = cancellation_token.cancelled() => { + tracing::info!("Received stop signal, stopping SynchronizationStateConsistencyChecker..."); } } } - Ok(()) }); } } diff --git a/nft_ingester/src/backfiller.rs b/nft_ingester/src/backfiller.rs index 640994fe9..b049c79c1 100644 --- a/nft_ingester/src/backfiller.rs +++ b/nft_ingester/src/backfiller.rs @@ -123,7 +123,7 @@ where } } pub async fn run_backfill_slots( - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, db: Arc, slot_db: Arc, consumer: Arc, @@ -132,13 +132,13 @@ pub async fn run_backfill_slots( C: BlockConsumer, { loop { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received, stopping run_backfill_slots"); break; } let sleep = tokio::time::sleep(Duration::from_millis(400)); if let Err(e) = backfill_slots( - &shutdown_token, + cancellation_token.child_token(), db.clone(), slot_db.clone(), consumer.clone(), @@ -150,7 +150,7 @@ pub async fn run_backfill_slots( } tokio::select! { _ = sleep => {} - _ = shutdown_token.cancelled() => { + _ = cancellation_token.cancelled() => { info!("Shutdown signal received, stopping run_backfill_slots"); break; } @@ -159,7 +159,7 @@ pub async fn run_backfill_slots( } pub async fn backfill_slots( - shutdown_token: &CancellationToken, + cancellation_token: CancellationToken, db: Arc, slot_db: Arc, consumer: Arc, @@ -182,7 +182,7 @@ where it.seek_to_first(); } while it.valid() { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received, stopping backfill_slots"); break; } diff --git a/nft_ingester/src/batch_mint/batch_mint_persister.rs b/nft_ingester/src/batch_mint/batch_mint_persister.rs index b09938bb2..5d1d5da78 100644 --- a/nft_ingester/src/batch_mint/batch_mint_persister.rs +++ b/nft_ingester/src/batch_mint/batch_mint_persister.rs @@ -9,7 +9,8 @@ use entities::{ use interface::{batch_mint::BatchMintDownloader, error::UsecaseError}; use metrics_utils::{BatchMintPersisterMetricsConfig, MetricStatus}; use rocks_db::columns::batch_mint::BatchMintWithStaker; -use tokio::{sync::broadcast::Receiver, task::JoinError, time::Instant}; +use tokio::{task::JoinError, time::Instant}; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::{ @@ -58,8 +59,11 @@ impl BatchMintPersister { Self { rocks_client, downloader, metrics } } - pub async fn persist_batch_mints(&self, mut rx: Receiver<()>) -> Result<(), JoinError> { - while rx.is_empty() { + pub async fn persist_batch_mints( + &self, + cancellation_token: CancellationToken, + ) -> Result<(), JoinError> { + while !cancellation_token.is_cancelled() { let (batch_mint_to_verify, batch_mint) = match self.get_batch_mint_to_verify().await { Ok(res) => res, Err(_) => { @@ -70,13 +74,18 @@ impl BatchMintPersister { // no batch_mints to persist continue; }; - self.persist_batch_mint(&rx, batch_mint_to_verify, batch_mint).await; + self.persist_batch_mint( + cancellation_token.child_token(), + batch_mint_to_verify, + batch_mint, + ) + .await; tokio::select! { - _ = tokio::time::sleep(Duration::from_secs(5)) => {}, - _ = rx.recv() => { - info!("Received stop signal, stopping ..."); - return Ok(()); - }, + _ = tokio::time::sleep(Duration::from_secs(5)) => {} + _ = cancellation_token.cancelled() => { + info!("Received stop signal, stopping ..."); + break; + } } } Ok(()) @@ -84,13 +93,13 @@ impl BatchMintPersister { pub async fn persist_batch_mint( &self, - rx: &Receiver<()>, + cancellation_token: CancellationToken, mut batch_mint_to_verify: BatchMintToVerify, mut batch_mint: Option>, ) { let start_time = Instant::now(); info!("Persisting {} batch_mint", &batch_mint_to_verify.url); - while rx.is_empty() { + while !cancellation_token.is_cancelled() { match &batch_mint_to_verify.persisting_state { &PersistingBatchMintState::ReceivedTransaction => { if let Err(err) = diff --git a/nft_ingester/src/batch_mint/batch_mint_processor.rs b/nft_ingester/src/batch_mint/batch_mint_processor.rs index b291dea21..d4b1d36e8 100644 --- a/nft_ingester/src/batch_mint/batch_mint_processor.rs +++ b/nft_ingester/src/batch_mint/batch_mint_processor.rs @@ -12,7 +12,8 @@ use metrics_utils::BatchMintProcessorMetricsConfig; use postgre_client::{model::BatchMintState, PgClient}; use rocks_db::{columns::batch_mint::BatchMintWithStaker, Storage}; use solana_program::pubkey::Pubkey; -use tokio::{sync::broadcast::Receiver, task::JoinError, time::Instant}; +use tokio::{task::JoinError, time::Instant}; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::error::IngesterError; @@ -28,10 +29,10 @@ const FILE_PROCESSING_METRICS_LABEL: &str = "batch_mint_file_processing"; pub async fn process_batch_mints( processor_clone: Arc>, - rx: Receiver<()>, + cancellation_token: CancellationToken, ) -> Result<(), JoinError> { info!("Start processing batch_mints..."); - processor_clone.process_batch_mints(rx).await; + processor_clone.process_batch_mints(cancellation_token).await; info!("Finish processing batch_mints..."); Ok(()) @@ -168,8 +169,8 @@ impl BatchMintProcessor { } } - pub async fn process_batch_mints(&self, mut rx: Receiver<()>) { - while rx.is_empty() { + pub async fn process_batch_mints(&self, cancellation_token: CancellationToken) { + while !cancellation_token.is_cancelled() { let batch_mint_to_process = match self.pg_client.fetch_batch_mint_for_processing().await { Ok(Some(batch_mint)) => batch_mint, @@ -181,12 +182,16 @@ impl BatchMintProcessor { continue; }, }; - if let Err(e) = self.process_batch_mint(rx.resubscribe(), batch_mint_to_process).await { + if let Err(e) = self + .process_batch_mint(cancellation_token.child_token(), batch_mint_to_process) + .await + { error!("process_batch_mint: {}", e); } + tokio::select! { _ = tokio::time::sleep(Duration::from_secs(5)) => {}, - _ = rx.recv() => { + _ = cancellation_token.cancelled() => { info!("Received stop signal, stopping ..."); return; }, @@ -196,7 +201,7 @@ impl BatchMintProcessor { pub async fn process_batch_mint( &self, - rx: Receiver<()>, + cancellation_token: CancellationToken, mut batch_mint_to_process: BatchMintWithState, ) -> Result<(), IngesterError> { info!("Processing {} batch_mint file", &batch_mint_to_process.file_name); @@ -204,7 +209,7 @@ impl BatchMintProcessor { let (batch_mint, file_size, file_checksum) = self.read_batch_mint_file(&batch_mint_to_process).await?; let mut metadata_url = String::new(); - while rx.is_empty() { + while !cancellation_token.is_cancelled() { match &batch_mint_to_process.state { entities::enums::BatchMintState::Uploaded => { self.process_batch_mint_validation(&batch_mint, &mut batch_mint_to_process) diff --git a/nft_ingester/src/bin/api/main.rs b/nft_ingester/src/bin/api/main.rs index 427ac4356..adc21397a 100644 --- a/nft_ingester/src/bin/api/main.rs +++ b/nft_ingester/src/bin/api/main.rs @@ -7,17 +7,13 @@ use nft_ingester::{ config::{init_logger, ApiClapArgs}, consts::RAYDIUM_API_HOST, error::IngesterError, - init::graceful_stop, json_worker::JsonWorker, raydium_price_fetcher::{RaydiumTokenPriceFetcher, CACHE_TTL}, }; use prometheus_client::registry::Registry; use rocks_db::{migrator::MigrationState, Storage}; use solana_client::nonblocking::rpc_client::RpcClient; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; +use tokio_util::sync::CancellationToken; use tracing::{error, info, warn}; use usecase::proofs::MaybeProofChecker; @@ -47,7 +43,7 @@ pub async fn main() -> Result<(), IngesterError> { let json_downloader_metrics = Arc::new(JsonDownloaderMetricsConfig::new()); json_downloader_metrics.register(&mut registry); - tokio::spawn(async move { + usecase::executor::spawn(async move { match setup_metrics(registry, args.metrics_port).await { Ok(_) => { info!("Setup metrics successfully") @@ -58,6 +54,26 @@ pub async fn main() -> Result<(), IngesterError> { } }); + let cancellation_token = CancellationToken::new(); + + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + // --stop + #[cfg(not(feature = "profiling"))] + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + + #[cfg(feature = "profiling")] + nft_ingester::init::graceful_stop( + cancellation_token, + guard, + args.profiling_file_path_container, + &args.heap_path, + ) + .await; + } + }); + let pg_client = postgre_client::PgClient::new( &args.pg_database_url, args.pg_min_db_connections, @@ -68,13 +84,10 @@ pub async fn main() -> Result<(), IngesterError> { ) .await?; let pg_client = Arc::new(pg_client); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let storage = Storage::open_secondary( &args.rocks_db_path, &args.rocks_db_secondary_path, - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) @@ -127,10 +140,6 @@ pub async fn main() -> Result<(), IngesterError> { } }; - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - let cloned_tasks = mutexed_tasks.clone(); - let cloned_rx = shutdown_rx.resubscribe(); - info!("Init token/price information..."); let token_price_fetcher = Arc::new(RaydiumTokenPriceFetcher::new( RAYDIUM_API_HOST.to_string(), @@ -138,49 +147,42 @@ pub async fn main() -> Result<(), IngesterError> { Some(red_metrics.clone()), )); let tpf = token_price_fetcher.clone(); - let tasks_clone = mutexed_tasks.clone(); - tasks_clone.lock().await.spawn(async move { + usecase::executor::spawn(async move { if let Err(e) = tpf.warmup().await { warn!(error = %e, "Failed to warm up Raydium token price fetcher, cache is empty: {:?}", e); } let (symbol_cache_size, _) = tpf.get_cache_sizes(); info!(%symbol_cache_size, "Warmed up Raydium token price fetcher with {} symbols", symbol_cache_size); - Ok(()) }); - mutexed_tasks.lock().await.spawn(async move { - match start_api( - pg_client.clone(), - cloned_rocks_storage.clone(), - cloned_rx, - metrics.clone(), - args.server_port, - proof_checker, - tree_gaps_checker, - args.max_page_limit, - json_worker, - None, - args.json_middleware_config.clone(), - cloned_tasks, - &args.rocks_archives_dir, - args.consistence_synchronization_api_threshold, - args.consistence_backfilling_slots_threshold, - args.batch_mint_service_port, - args.file_storage_path_container.as_str(), - account_balance_getter, - args.storage_service_base_url, - args.native_mint_pubkey, - token_price_fetcher, - ) - .await - { - Ok(_) => Ok(()), - Err(e) => { - error!("Start API: {}", e); - // cannot return JointError here - Ok(()) - }, + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + let _ = start_api( + pg_client.clone(), + cloned_rocks_storage.clone(), + cancellation_token, + metrics.clone(), + args.server_port, + proof_checker, + tree_gaps_checker, + args.max_page_limit, + json_worker, + None, + args.json_middleware_config.clone(), + &args.rocks_archives_dir, + args.consistence_synchronization_api_threshold, + args.consistence_backfilling_slots_threshold, + args.batch_mint_service_port, + args.file_storage_path_container.as_str(), + account_balance_getter, + args.storage_service_base_url, + args.native_mint_pubkey, + token_price_fetcher, + ) + .await + .inspect_err(|e| error!(error = %e, "Start API: {}", e)); } }); @@ -216,34 +218,22 @@ pub async fn main() -> Result<(), IngesterError> { // }); // try synchronizing secondary rocksdb instance every config.rocks_sync_interval_seconds - let cloned_rx = shutdown_rx.resubscribe(); - let cloned_rocks_storage = rocks_storage.clone(); let dur = tokio::time::Duration::from_secs(args.rocks_sync_interval_seconds); - mutexed_tasks.lock().await.spawn(async move { - while cloned_rx.is_empty() { - if let Err(e) = cloned_rocks_storage.db.try_catch_up_with_primary() { - error!("Sync rocksdb error: {}", e); + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + let rocks_storage = rocks_storage.clone(); + async move { + while !cancellation_token.is_cancelled() { + if let Err(e) = rocks_storage.db.try_catch_up_with_primary() { + error!("Sync rocksdb error: {}", e); + } + tokio::time::sleep(dur).await; } - tokio::time::sleep(dur).await; } - - Ok(()) }); - // --stop - #[cfg(not(feature = "profiling"))] - graceful_stop(mutexed_tasks, shutdown_tx, None).await; - - #[cfg(feature = "profiling")] - graceful_stop( - mutexed_tasks, - shutdown_tx, - None, - guard, - args.profiling_file_path_container, - &args.heap_path, - ) - .await; - + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } Ok(()) } diff --git a/nft_ingester/src/bin/backfill/main.rs b/nft_ingester/src/bin/backfill/main.rs index 2f03a5c5d..0c1741ca1 100644 --- a/nft_ingester/src/bin/backfill/main.rs +++ b/nft_ingester/src/bin/backfill/main.rs @@ -62,16 +62,18 @@ async fn main() { Storage::open_readonly_with_cfs_only_db(&args.source_db_path, SlotStorage::cf_names()) .expect("Failed to open source RocksDB"); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); + let cancellation_token = CancellationToken::new(); + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + } + }); // Open target RocksDB let target_db = Arc::new( - Storage::open( - &args.target_db_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .expect("Failed to open target RocksDB"), + Storage::open(&args.target_db_path, red_metrics.clone(), MigrationState::Last) + .expect("Failed to open target RocksDB"), ); // Initialize the DirectBlockParser @@ -93,23 +95,22 @@ async fn main() { let slots_processed = Arc::new(AtomicU64::new(0)); let rate = Arc::new(Mutex::new(0.0)); - // Spawn a task to handle graceful shutdown on Ctrl+C - let shutdown_token = CancellationToken::new(); - let shutdown_token_clone = shutdown_token.clone(); - - let slot_sender_clone = slot_sender.clone(); - tokio::spawn(async move { - // Wait for Ctrl+C signal - match tokio::signal::ctrl_c().await { - Ok(()) => { - info!("Received Ctrl+C, shutting down gracefully..."); - shutdown_token_clone.cancel(); - // Close the channel to signal workers to stop - slot_sender_clone.close(); - }, - Err(err) => { - error!("Unable to listen for shutdown signal: {}", err); - }, + usecase::executor::spawn({ + let cancellation_token = cancellation_token.clone(); + let slot_sender = slot_sender.clone(); + async move { + // Wait for Ctrl+C signal + match tokio::signal::ctrl_c().await { + Ok(()) => { + info!("Received Ctrl+C, shutting down gracefully..."); + cancellation_token.cancel(); + // Close the channel to signal workers to stop + slot_sender.close(); + }, + Err(err) => { + error!("Unable to listen for shutdown signal: {}", err); + }, + } } }); @@ -197,82 +198,85 @@ async fn main() { let progress_bar = progress_bar.clone(); let slots_processed = slots_processed.clone(); let rate = rate.clone(); - let shutdown_token = shutdown_token.clone(); let slot_receiver = slot_receiver.clone(); - let handle = tokio::spawn(async move { - while let Ok((slot, raw_block_data)) = slot_receiver.recv().await { - if shutdown_token.is_cancelled() { - break; - } - - // Process the slot - let raw_block: RawBlock = match RawBlock::decode(&raw_block_data) { - Ok(rb) => rb, - Err(e) => { - error!("Failed to decode the value for slot {}: {}", slot, e); - continue; - }, - }; - - if let Err(e) = consumer.consume_block(slot, raw_block.block).await { - error!("Error processing slot {}: {}", slot, e); + let handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + while let Ok((slot, raw_block_data)) = slot_receiver.recv().await { + if cancellation_token.is_cancelled() { + break; + } + + // Process the slot + let raw_block: RawBlock = match RawBlock::decode(&raw_block_data) { + Ok(rb) => rb, + Err(e) => { + error!("Failed to decode the value for slot {}: {}", slot, e); + continue; + }, + }; + + if let Err(e) = consumer.consume_block(slot, raw_block.block).await { + error!("Error processing slot {}: {}", slot, e); + } + + // Increment slots_processed + let current_slots_processed = + slots_processed.fetch_add(1, Ordering::Relaxed) + 1; + + // Update progress bar position and message + progress_bar.inc(1); + + let current_rate = { + let rate_guard = rate.lock().unwrap(); + *rate_guard + }; + progress_bar.set_message(format!( + "Slots Processed: {} Current Slot: {} Rate: {:.2}/s", + current_slots_processed, slot, current_rate + )); } - - // Increment slots_processed - let current_slots_processed = slots_processed.fetch_add(1, Ordering::Relaxed) + 1; - - // Update progress bar position and message - progress_bar.inc(1); - - let current_rate = { - let rate_guard = rate.lock().unwrap(); - *rate_guard - }; - progress_bar.set_message(format!( - "Slots Processed: {} Current Slot: {} Rate: {:.2}/s", - current_slots_processed, slot, current_rate - )); } }); worker_handles.push(handle); } - // Spawn a task to update the rate periodically - let slots_processed_clone = slots_processed.clone(); - let rate_clone = rate.clone(); - let shutdown_token_clone = shutdown_token.clone(); + usecase::executor::spawn({ + let cancellation_token = cancellation_token.clone(); + let slots_processed = slots_processed.clone(); + let rate = rate.clone(); + async move { + let mut last_time = std::time::Instant::now(); + let mut last_count = slots_processed.load(Ordering::Relaxed); - tokio::spawn(async move { - let mut last_time = std::time::Instant::now(); - let mut last_count = slots_processed_clone.load(Ordering::Relaxed); + loop { + tokio::time::sleep(std::time::Duration::from_secs(1)).await; - loop { - tokio::time::sleep(std::time::Duration::from_secs(1)).await; + if cancellation_token.is_cancelled() { + break; + } - if shutdown_token_clone.is_cancelled() { - break; - } + let current_time = std::time::Instant::now(); + let current_count = slots_processed.load(Ordering::Relaxed); - let current_time = std::time::Instant::now(); - let current_count = slots_processed_clone.load(Ordering::Relaxed); + let elapsed = current_time.duration_since(last_time).as_secs_f64(); + let count = current_count - last_count; - let elapsed = current_time.duration_since(last_time).as_secs_f64(); - let count = current_count - last_count; + let current_rate = if elapsed > 0.0 { (count as f64) / elapsed } else { 0.0 }; - let current_rate = if elapsed > 0.0 { (count as f64) / elapsed } else { 0.0 }; + // Update rate + { + let mut rate_guard = rate.lock().unwrap(); + *rate_guard = current_rate; + } - // Update rate - { - let mut rate_guard = rate_clone.lock().unwrap(); - *rate_guard = current_rate; + // Update for next iteration + last_time = current_time; + last_count = current_count; } - - // Update for next iteration - last_time = current_time; - last_count = current_count; } }); @@ -283,7 +287,7 @@ async fn main() { slots_to_process, source_db, slot_sender.clone(), - shutdown_token.clone(), + cancellation_token.child_token(), ) .await; } else { @@ -291,7 +295,7 @@ async fn main() { send_all_slots_to_workers( source_db, slot_sender.clone(), - shutdown_token.clone(), + cancellation_token.child_token(), args.first_slot, args.last_slot, ) @@ -306,6 +310,9 @@ async fn main() { let _ = handle.await; } + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } progress_bar.finish_with_message("Processing complete"); } @@ -314,12 +321,12 @@ async fn send_slots_to_workers( slots_to_process: Vec, source_db: rocksdb::DB, slot_sender: async_channel::Sender<(u64, Vec)>, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, ) { let cf_handle = source_db.cf_handle(RawBlock::NAME).unwrap(); for slot in slots_to_process { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received. Stopping the submission of new slots."); break; } @@ -347,7 +354,7 @@ async fn send_slots_to_workers( async fn send_all_slots_to_workers( source_db: rocksdb::DB, slot_sender: async_channel::Sender<(u64, Vec)>, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, first_slot: Option, last_slot: Option, ) { @@ -367,7 +374,7 @@ async fn send_all_slots_to_workers( // Send slots to the channel while iter.valid() { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received. Stopping the submission of new slots."); break; } diff --git a/nft_ingester/src/bin/burnt_assets_ingester/main.rs b/nft_ingester/src/bin/burnt_assets_ingester/main.rs index 3f6f9971e..12dda4e51 100644 --- a/nft_ingester/src/bin/burnt_assets_ingester/main.rs +++ b/nft_ingester/src/bin/burnt_assets_ingester/main.rs @@ -294,13 +294,8 @@ async fn main() { // Open target RocksDB let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let target_db = Arc::new( - Storage::open( - &args.target_db_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .expect("Failed to open target RocksDB"), + Storage::open(&args.target_db_path, red_metrics.clone(), MigrationState::Last) + .expect("Failed to open target RocksDB"), ); // Initialize metrics diff --git a/nft_ingester/src/bin/dumper/main.rs b/nft_ingester/src/bin/dumper/main.rs index 8ae963012..79edb234f 100644 --- a/nft_ingester/src/bin/dumper/main.rs +++ b/nft_ingester/src/bin/dumper/main.rs @@ -2,16 +2,15 @@ use std::{fs::File, path::PathBuf, sync::Arc}; use clap::{command, Parser}; use metrics_utils::SynchronizerMetricsConfig; -use nft_ingester::{error::IngesterError, index_syncronizer::shard_pubkeys, init::graceful_stop}; +use nft_ingester::{error::IngesterError, index_synchronizer::shard_pubkeys}; use rocks_db::{ migrator::MigrationState, storage_traits::{AssetUpdateIndexStorage, Dumper}, Storage, }; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; +use tokio::task::JoinSet; +use tokio_util::sync::CancellationToken; +use tracing::error; #[derive(Parser, Debug)] #[command(author, version, about, long_about = None)] @@ -63,31 +62,26 @@ pub async fn main() -> Result<(), IngesterError> { let metrics = Arc::new(SynchronizerMetricsConfig::new()); let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - + let cancellation_token = CancellationToken::new(); + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + #[cfg(not(feature = "profiling"))] + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + #[cfg(feature = "profiling")] + nft_ingester::init::graceful_stop(cancellation_token, None, None, "").await; + } + }); let rocks_storage = Arc::new( Storage::open_secondary( &args.source_path, &secondary_storage_path, - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) .unwrap(), ); - let cloned_tasks = mutexed_tasks.clone(); - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - mutexed_tasks.lock().await.spawn(async move { - // --stop - #[cfg(not(feature = "profiling"))] - graceful_stop(cloned_tasks, shutdown_tx, None).await; - #[cfg(feature = "profiling")] - graceful_stop(cloned_tasks, shutdown_tx, None, None, None, "").await; - - Ok(()) - }); if let Err(e) = rocks_storage.db.try_catch_up_with_primary() { tracing::error!("Sync rocksdb error: {}", e); } @@ -154,22 +148,24 @@ pub async fn main() -> Result<(), IngesterError> { let start = *start; let end = *end; - let shutdown_rx = shutdown_rx.resubscribe(); let metrics = metrics.clone(); let rocks_storage = rocks_storage.clone(); - tasks.spawn_blocking(move || { - rocks_storage.dump_nft_csv( - assets_file, - creators_file, - authority_file, - metadata_file, - args.buffer_capacity, - args.limit, - Some(start), - Some(end), - &shutdown_rx, - metrics, - ) + tasks.spawn_blocking({ + let cancellation_token = cancellation_token.child_token(); + move || { + rocks_storage.dump_nft_csv( + assets_file, + creators_file, + authority_file, + metadata_file, + args.buffer_capacity, + args.limit, + Some(start), + Some(end), + cancellation_token, + metrics, + ) + } }); } @@ -190,18 +186,20 @@ pub async fn main() -> Result<(), IngesterError> { let start = *start; let end = *end; - let shutdown_rx = shutdown_rx.resubscribe(); let metrics = metrics.clone(); let rocks_storage = rocks_storage.clone(); - fungible_tasks.spawn_blocking(move || { - rocks_storage.dump_fungible_csv( - (fungible_tokens_file, fungible_tokens_path), - args.buffer_capacity, - Some(start), - Some(end), - &shutdown_rx, - metrics, - ) + fungible_tasks.spawn_blocking({ + let cancellation_token = cancellation_token.child_token(); + move || { + rocks_storage.dump_fungible_csv( + (fungible_tokens_file, fungible_tokens_path), + args.buffer_capacity, + Some(start), + Some(end), + cancellation_token, + metrics, + ) + } }); } @@ -224,5 +222,8 @@ pub async fn main() -> Result<(), IngesterError> { tracing::info!("Dumping fungible tokens done"); let keys_file = File::create(base_path.join("keys.csv")).expect("should create keys file"); Storage::dump_last_keys(keys_file, last_known_key, last_known_fungible_key)?; + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } Ok(()) } diff --git a/nft_ingester/src/bin/ingester/main.rs b/nft_ingester/src/bin/ingester/main.rs index 0acb49af3..215ad16c4 100644 --- a/nft_ingester/src/bin/ingester/main.rs +++ b/nft_ingester/src/bin/ingester/main.rs @@ -14,7 +14,6 @@ use arweave_rs::{consts::ARWEAVE_BASE_URL, Arweave}; use backfill_rpc::rpc::BackfillRPC; use clap::Parser; use entities::enums::ASSET_TYPES; -use futures::FutureExt; use grpc::{ asseturls::asset_url_service_server::AssetUrlServiceServer, asseturls_impl::AssetUrlServiceImpl, client::Client, @@ -37,7 +36,7 @@ use nft_ingester::{ consts::RAYDIUM_API_HOST, error::IngesterError, gapfiller::{process_asset_details_stream_wrapper, run_sequence_consistent_gapfiller}, - init::{graceful_stop, init_index_storage_with_migration, init_primary_storage}, + init::{init_index_storage_with_migration, init_primary_storage}, json_worker, json_worker::JsonWorker, processors::{ @@ -57,11 +56,6 @@ use postgre_client::PG_MIGRATIONS_PATH; use pprof::ProfilerGuardBuilder; use rocks_db::{storage_traits::AssetSlotStorage, SlotStorage}; use solana_client::nonblocking::rpc_client::RpcClient; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, - time::sleep as tokio_sleep, -}; use tokio_util::sync::CancellationToken; use tonic::transport::Server; use tracing::{error, info, warn}; @@ -114,6 +108,25 @@ pub async fn main() -> Result<(), IngesterError> { .expect("Failed to build 'ProfilerGuardBuilder'!") }); + let cancellation_token = CancellationToken::new(); + + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + // --stop + #[cfg(not(feature = "profiling"))] + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + + #[cfg(feature = "profiling")] + nft_ingester::init::graceful_stop( + cancellation_token, + guard, + args.profiling_file_path_container, + &args.heap_path, + ) + .await; + } + }); // try to restore rocksDB first if args.is_restore_rocks_db { restore_rocksdb( @@ -131,9 +144,6 @@ pub async fn main() -> Result<(), IngesterError> { .await?; } - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - info!("Init primary storage..."); let primary_rocks_storage = Arc::new( init_primary_storage( @@ -141,7 +151,6 @@ pub async fn main() -> Result<(), IngesterError> { args.enable_rocks_migration.unwrap_or(false), &args.rocks_migration_storage_path, &metrics_state, - mutexed_tasks.clone(), ) .await?, ); @@ -173,28 +182,25 @@ pub async fn main() -> Result<(), IngesterError> { Some(metrics_state.red_metrics.clone()), )); let tpf = token_price_fetcher.clone(); - let tasks_clone = mutexed_tasks.clone(); - tasks_clone.lock().await.spawn(async move { + cancellation_token.run_until_cancelled(async move { if let Err(e) = tpf.warmup().await { warn!(error = %e, "Failed to warm up Raydium token price fetcher, cache is empty: {:?}", e); } let (symbol_cache_size, _) = tpf.get_cache_sizes(); info!(%symbol_cache_size, "Warmed up Raydium token price fetcher with {} symbols", symbol_cache_size); - Ok(()) - }); + }).await; let well_known_fungible_accounts = token_price_fetcher.get_all_token_symbols().await.unwrap_or_else(|_| HashMap::new()); info!("Init Redis ...."); - let cloned_rx = shutdown_rx.resubscribe(); let message_config = MessengerConfig { messenger_type: MessengerType::Redis, connection_config: args.redis_connection_config.clone(), }; let ack_channel = - create_ack_channel(cloned_rx, message_config.clone(), mutexed_tasks.clone()).await; + create_ack_channel(message_config.clone(), cancellation_token.child_token()).await; for index in 0..args.redis_accounts_parsing_workers { let account_consumer_worker_name = Uuid::new_v4().to_string(); @@ -226,8 +232,7 @@ pub async fn main() -> Result<(), IngesterError> { ); run_accounts_processor( - shutdown_rx.resubscribe(), - mutexed_tasks.clone(), + cancellation_token.child_token(), redis_receiver, primary_rocks_storage.clone(), args.account_processor_buffer_size, @@ -236,11 +241,9 @@ pub async fn main() -> Result<(), IngesterError> { Some(metrics_state.message_process_metrics.clone()), index_pg_storage.clone(), rpc_client.clone(), - mutexed_tasks.clone(), Some(account_consumer_worker_name.clone()), well_known_fungible_accounts.clone(), - ) - .await; + ); } for index in 0..args.redis_transactions_parsing_workers { @@ -272,13 +275,11 @@ pub async fn main() -> Result<(), IngesterError> { ); run_transaction_processor( - shutdown_rx.resubscribe(), - mutexed_tasks.clone(), + cancellation_token.child_token(), redis_receiver, geyser_bubblegum_updates_processor.clone(), Some(metrics_state.message_process_metrics.clone()), - ) - .await; + ); } info!("MessageSource Redis FINISH"); @@ -288,12 +289,11 @@ pub async fn main() -> Result<(), IngesterError> { let first_processed_slot = Arc::new(AtomicU64::new(0)); let first_processed_slot_clone = first_processed_slot.clone(); let cloned_rocks_storage = primary_rocks_storage.clone(); - let cloned_rx = shutdown_rx.resubscribe(); - let cloned_tx = shutdown_tx.clone(); - mutexed_tasks.lock().await.spawn(receive_last_saved_slot( - cloned_rx, - cloned_tx, + usecase::executor::spawn(receive_last_saved_slot( + // NOTE: the clone here is important to bubble up the cancellation + // from this function to other child tokens. + cancellation_token.clone(), cloned_rocks_storage, first_processed_slot_clone, last_saved_slot, @@ -319,18 +319,19 @@ pub async fn main() -> Result<(), IngesterError> { .map_err(|e| error!("GRPC Client new: {e}")) .expect("Failed to create GRPC Client"); - while first_processed_slot.load(Ordering::Relaxed) == 0 && shutdown_rx.is_empty() { - tokio_sleep(Duration::from_millis(100)).await + while first_processed_slot.load(Ordering::Relaxed) == 0 + && !cancellation_token.is_cancelled() + { + tokio::time::sleep(Duration::from_millis(100)).await } let cloned_rocks_storage = primary_rocks_storage.clone(); - if shutdown_rx.is_empty() { + if !cancellation_token.is_cancelled() { let gaped_data_client_clone = gaped_data_client.clone(); let first_processed_slot_value = first_processed_slot.load(Ordering::Relaxed); - let cloned_rx = shutdown_rx.resubscribe(); - mutexed_tasks.lock().await.spawn(process_asset_details_stream_wrapper( - cloned_rx, + usecase::executor::spawn(process_asset_details_stream_wrapper( + cancellation_token.child_token(), cloned_rocks_storage, last_saved_slot, first_processed_slot_value, @@ -339,9 +340,8 @@ pub async fn main() -> Result<(), IngesterError> { )); let cloned_rocks_storage = primary_rocks_storage.clone(); - let cloned_rx = shutdown_rx.resubscribe(); - mutexed_tasks.lock().await.spawn(process_asset_details_stream_wrapper( - cloned_rx, + usecase::executor::spawn(process_asset_details_stream_wrapper( + cancellation_token.child_token(), cloned_rocks_storage, last_saved_slot, first_processed_slot_value, @@ -359,10 +359,9 @@ pub async fn main() -> Result<(), IngesterError> { args.check_proofs_probability, args.check_proofs_commitment, ))); - let cloned_rx = shutdown_rx.resubscribe(); let file_storage_path = args.file_storage_path_container.clone(); - if args.run_api.unwrap_or(false) { + if args.run_api.unwrap_or_default() { info!("Starting API (Ingester)..."); let middleware_json_downloader = args .json_middleware_config @@ -382,49 +381,44 @@ pub async fn main() -> Result<(), IngesterError> { let cloned_index_storage = index_pg_storage.clone(); - mutexed_tasks.lock().await.spawn(async move { - match start_api( - cloned_index_storage, - cloned_rocks_storage.clone(), - cloned_rx, - cloned_api_metrics, - args.server_port, - proof_checker, - tree_gaps_checker, - args.max_page_limit, - middleware_json_downloader.clone(), - middleware_json_downloader, - args.json_middleware_config, - tasks_clone, - &args.archives_dir, - args.consistence_synchronization_api_threshold, - args.consistence_backfilling_slots_threshold, - args.batch_mint_service_port, - args.file_storage_path_container.as_str(), - account_balance_getter, - args.storage_service_base_url, - args.native_mint_pubkey, - token_price_fetcher, - ) - .await - { - Ok(_) => Ok(()), - Err(e) => { - error!("Start API: {}", e); - Ok(()) - }, + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + start_api( + cloned_index_storage, + cloned_rocks_storage.clone(), + cancellation_token, + cloned_api_metrics, + args.server_port, + proof_checker, + tree_gaps_checker, + args.max_page_limit, + middleware_json_downloader.clone(), + middleware_json_downloader, + args.json_middleware_config, + &args.archives_dir, + args.consistence_synchronization_api_threshold, + args.consistence_backfilling_slots_threshold, + args.batch_mint_service_port, + args.file_storage_path_container.as_str(), + account_balance_getter, + args.storage_service_base_url, + args.native_mint_pubkey, + token_price_fetcher, + ) + .await + .inspect_err(|e| error!("Start API: {}", e)) } }); } - let cloned_rx = shutdown_rx.resubscribe(); - let cloned_jp = json_processor.clone(); - mutexed_tasks.lock().await.spawn(json_worker::run(cloned_jp, cloned_rx).map(|_| Ok(()))); - - let shutdown_token = CancellationToken::new(); + usecase::executor::spawn(json_worker::run( + json_processor.clone(), + cancellation_token.child_token(), + )); // Backfiller - if args.run_backfiller.unwrap_or(false) { + if args.run_backfiller.unwrap_or_default() { info!("Start backfiller..."); let backfill_bubblegum_updates_processor = Arc::new(BubblegumTxProcessor::new( @@ -441,7 +435,6 @@ pub async fn main() -> Result<(), IngesterError> { .clone() .expect("slots_db_path is required for SlotStorage"), args.rocks_secondary_slots_db_path.clone(), - mutexed_tasks.clone(), metrics_state.red_metrics.clone(), ) .expect("Failed to open slot storage"), @@ -459,7 +452,7 @@ pub async fn main() -> Result<(), IngesterError> { .await, ); - if args.run_bubblegum_backfiller.unwrap_or(false) { + if args.run_bubblegum_backfiller.unwrap_or_default() { info!("Runing Bubblegum backfiller (ingester)..."); if args.should_reingest { @@ -476,20 +469,21 @@ pub async fn main() -> Result<(), IngesterError> { primary_rocks_storage.clone(), metrics_state.backfiller_metrics.clone(), )); - let shutdown_token = shutdown_token.clone(); let db = primary_rocks_storage.clone(); let metrics: Arc = metrics_state.backfiller_metrics.clone(); let slot_db = slot_db.clone(); - mutexed_tasks.lock().await.spawn(async move { - nft_ingester::backfiller::run_backfill_slots( - shutdown_token, - db, - slot_db, - consumer, - metrics, - ) - .await; - Ok(()) + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + nft_ingester::backfiller::run_backfill_slots( + cancellation_token, + db, + slot_db, + consumer, + metrics, + ) + .await; + } }); } @@ -508,9 +502,8 @@ pub async fn main() -> Result<(), IngesterError> { metrics_state.sequence_consistent_gapfill_metrics.clone(), backfiller_source.clone(), direct_block_parser, - shutdown_rx.resubscribe(), + cancellation_token.child_token(), rpc_backfiller.clone(), - mutexed_tasks.clone(), args.sequence_consistent_checker_wait_period_sec, ) .await; @@ -527,19 +520,19 @@ pub async fn main() -> Result<(), IngesterError> { let asset_url_serv = AssetUrlServiceImpl::new(primary_rocks_storage.clone()); let addr = format!("0.0.0.0:{}", args.peer_grpc_port).parse()?; // Spawn the gRPC server task and add to JoinSet - let mut rx = shutdown_rx.resubscribe(); - mutexed_tasks.lock().await.spawn(async move { - if let Err(e) = Server::builder() - .add_service(GapFillerServiceServer::new(serv)) - .add_service(AssetUrlServiceServer::new(asset_url_serv)) - .serve_with_shutdown(addr, rx.recv().map(|_| ())) - .await - { - error!("Server error: {}", e); + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + if let Err(e) = Server::builder() + .add_service(GapFillerServiceServer::new(serv)) + .add_service(AssetUrlServiceServer::new(asset_url_serv)) + .serve_with_shutdown(addr, cancellation_token.cancelled()) + .await + { + error!("Server error: {}", e); + } } - - Ok(()) }); let rocks_clone = primary_rocks_storage.clone(); @@ -549,42 +542,55 @@ pub async fn main() -> Result<(), IngesterError> { tx_ingester.clone(), metrics_state.rpc_backfiller_metrics.clone(), ); - let cloned_rx = shutdown_rx.resubscribe(); let metrics_clone = metrics_state.rpc_backfiller_metrics.clone(); - mutexed_tasks.lock().await.spawn(async move { - let program_id = mpl_bubblegum::programs::MPL_BUBBLEGUM_ID; - while cloned_rx.is_empty() { - match signature_fetcher - .fetch_signatures(program_id, args.rpc_retry_interval_millis) - .await - { - Ok(_) => { - metrics_clone - .inc_run_fetch_signatures("fetch_signatures", MetricStatus::SUCCESS); - info!( - "signatures sync finished successfully for program_id: {}", - program_id - ); - }, - Err(e) => { - metrics_clone - .inc_run_fetch_signatures("fetch_signatures", MetricStatus::FAILURE); - error!("signatures sync failed: {:?} for program_id: {}", e, program_id); - }, - } + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + let program_id = mpl_bubblegum::programs::MPL_BUBBLEGUM_ID; + while !cancellation_token.is_cancelled() { + match signature_fetcher + .fetch_signatures(program_id, args.rpc_retry_interval_millis) + .await + { + Ok(_) => { + metrics_clone.inc_run_fetch_signatures( + "fetch_signatures", + MetricStatus::SUCCESS, + ); + info!( + "signatures sync finished successfully for program_id: {}", + program_id + ); + }, + Err(e) => { + metrics_clone.inc_run_fetch_signatures( + "fetch_signatures", + MetricStatus::FAILURE, + ); + error!( + "signatures sync failed: {:?} for program_id: {}", + e, program_id + ); + }, + } - tokio_sleep(Duration::from_secs(60)).await; + tokio::select! { + _ = cancellation_token.cancelled() => {} + _ = tokio::time::sleep(Duration::from_secs(60)) => {} + } + } } - - Ok(()) }); } - Scheduler::run_in_background(Scheduler::new( - primary_rocks_storage.clone(), - Some(well_known_fungible_accounts.keys().cloned().collect()), - )) + Scheduler::run_in_background( + Scheduler::new( + primary_rocks_storage.clone(), + Some(well_known_fungible_accounts.keys().cloned().collect()), + ), + cancellation_token.child_token(), + ) .await; if let Ok(arweave) = Arweave::from_keypair_path( @@ -602,9 +608,11 @@ pub async fn main() -> Result<(), IngesterError> { file_storage_path, metrics_state.batch_mint_processor_metrics.clone(), )); - let rx = shutdown_rx.resubscribe(); let processor_clone = batch_mint_processor.clone(); - mutexed_tasks.lock().await.spawn(process_batch_mints(processor_clone, rx)); + usecase::executor::spawn(process_batch_mints( + processor_clone, + cancellation_token.child_token(), + )); } let batch_mint_persister = BatchMintPersister::new( @@ -613,10 +621,12 @@ pub async fn main() -> Result<(), IngesterError> { metrics_state.batch_mint_persisting_metrics.clone(), ); - let rx = shutdown_rx.resubscribe(); - mutexed_tasks.lock().await.spawn(async move { - info!("Start batch_mint persister..."); - batch_mint_persister.persist_batch_mints(rx).await + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + info!("Start batch_mint persister..."); + batch_mint_persister.persist_batch_mints(cancellation_token).await + } }); // clean indexes @@ -624,47 +634,35 @@ pub async fn main() -> Result<(), IngesterError> { info!("Start cleaning index {:?}", asset_type); let primary_rocks_storage = primary_rocks_storage.clone(); - let mut rx = shutdown_rx.resubscribe(); let index_pg_storage = index_pg_storage.clone(); - mutexed_tasks.lock().await.spawn(async move { - let index_pg_storage = index_pg_storage.clone(); - tokio::select! { - _ = rx.recv() => {} - _ = async move { - loop { - match clean_syncronized_idxs(index_pg_storage.clone(), primary_rocks_storage.clone(), asset_type).await { - Ok(_) => { - info!("Cleaned synchronized indexes for {:?}", asset_type); - } - Err(e) => { - error!("Failed to clean synchronized indexes for {:?} with error {}", asset_type, e); + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + let index_pg_storage = index_pg_storage.clone(); + tokio::select! { + _ = cancellation_token.cancelled() => {} + _ = async move { + loop { + match clean_syncronized_idxs(index_pg_storage.clone(), primary_rocks_storage.clone(), asset_type).await { + Ok(_) => { + info!("Cleaned synchronized indexes for {:?}", asset_type); + } + Err(e) => { + error!("Failed to clean synchronized indexes for {:?} with error {}", asset_type, e); + } } + tokio::time::sleep(Duration::from_secs(SECONDS_TO_RETRY_IDXS_CLEANUP)).await; } - tokio::time::sleep(Duration::from_secs(SECONDS_TO_RETRY_IDXS_CLEANUP)).await; - } - } => {} + } => {} + } } - - Ok(()) }); } start_metrics(metrics_state.registry, args.metrics_port).await; - - // --stop - #[cfg(not(feature = "profiling"))] - graceful_stop(mutexed_tasks, shutdown_tx, Some(shutdown_token)).await; - - #[cfg(feature = "profiling")] - graceful_stop( - mutexed_tasks, - shutdown_tx, - Some(shutdown_token), - guard, - args.profiling_file_path_container, - &args.heap_path, - ) - .await; + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } Ok(()) } diff --git a/nft_ingester/src/bin/migrator/main.rs b/nft_ingester/src/bin/migrator/main.rs index 2007fabdb..fdfcdaed1 100644 --- a/nft_ingester/src/bin/migrator/main.rs +++ b/nft_ingester/src/bin/migrator/main.rs @@ -9,7 +9,6 @@ use metrics_utils::{ use nft_ingester::{ config::{init_logger, JsonMigratorMode, MigratorClapArgs}, error::IngesterError, - init::graceful_stop, }; use postgre_client::PgClient; use rocks_db::{ @@ -19,10 +18,8 @@ use rocks_db::{ migrator::MigrationState, Storage, }; -use tokio::{ - sync::{broadcast, broadcast::Receiver, Mutex}, - task::{JoinError, JoinSet}, -}; +use tokio::sync::Mutex; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; pub const DEFAULT_MIN_POSTGRES_CONNECTIONS: u32 = 100; @@ -52,12 +49,20 @@ pub async fn main() -> Result<(), IngesterError> { start_metrics(metrics_state.registry, args.metrics_port).await; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); + let cancellation_token = CancellationToken::new(); + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + #[cfg(not(feature = "profiling"))] + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + #[cfg(feature = "profiling")] + nft_ingester::init::graceful_stop(cancellation_token, None, None, "").await; + } + }); let storage = Storage::open( args.rocks_json_target_db.clone(), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, )?; @@ -65,7 +70,6 @@ pub async fn main() -> Result<(), IngesterError> { let target_storage = Arc::new(storage); let source_storage = Storage::open( args.rocks_json_source_db.clone(), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, )?; @@ -79,20 +83,16 @@ pub async fn main() -> Result<(), IngesterError> { args.migrator_mode, ); - let cloned_tasks = mutexed_tasks.clone(); - - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - - let cloned_rx = shutdown_rx.resubscribe(); - mutexed_tasks.lock().await.spawn(async move { - json_migrator.run(cloned_rx, cloned_tasks).await; - Ok(()) + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + json_migrator.run(cancellation_token).await; + } }); - #[cfg(not(feature = "profiling"))] - graceful_stop(mutexed_tasks, shutdown_tx, None).await; - #[cfg(feature = "profiling")] - graceful_stop(mutexed_tasks, shutdown_tx, None, None, None, "").await; + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } Ok(()) } @@ -116,40 +116,40 @@ impl JsonMigrator { Self { database_pool, source_rocks_db, target_rocks_db, metrics, migrator_mode } } - pub async fn run(&self, rx: Receiver<()>, tasks: Arc>>>) { + pub async fn run(&self, cancellation_token: CancellationToken) { match self.migrator_mode { JsonMigratorMode::Full => { info!("Launch JSON migrator in full mode"); info!("Start migrate JSONs..."); - self.migrate_jsons(rx.resubscribe()).await; + self.migrate_jsons(cancellation_token.child_token()).await; info!("JSONs are migrated. Start setting tasks..."); - self.set_tasks(rx.resubscribe(), tasks).await; + self.set_tasks(cancellation_token).await; info!("Tasks are set!"); }, JsonMigratorMode::JsonsOnly => { info!("Launch JSON migrator in jsons only mode"); info!("Start migrate JSONs..."); - self.migrate_jsons(rx.resubscribe()).await; + self.migrate_jsons(cancellation_token).await; info!("JSONs are migrated!"); }, JsonMigratorMode::TasksOnly => { info!("Launch JSON migrator in tasks only mode"); info!("Start set tasks..."); - self.set_tasks(rx.resubscribe(), tasks).await; + self.set_tasks(cancellation_token).await; info!("Tasks are set!"); }, } } - pub async fn migrate_jsons(&self, rx: Receiver<()>) { + pub async fn migrate_jsons(&self, cancellation_token: CancellationToken) { let all_available_jsons = self.source_rocks_db.asset_offchain_data.iter_end(); for json in all_available_jsons { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("JSON migrator is stopped"); break; } @@ -189,11 +189,7 @@ impl JsonMigrator { } } - pub async fn set_tasks( - &self, - rx: Receiver<()>, - tasks: Arc>>>, - ) { + pub async fn set_tasks(&self, cancellation_token: CancellationToken) { let mut assets_iter = self.target_rocks_db.db.raw_iterator_cf( &self.target_rocks_db.db.cf_handle(AssetCompleteDetails::NAME).unwrap(), ); @@ -206,61 +202,62 @@ impl JsonMigrator { let tasks_batch_to_insert = 1000; - let cloned_rx = rx.resubscribe(); - - tasks.lock().await.spawn(async move { - loop { - if !cloned_rx.is_empty() { - info!("Worker to clean tasks buffer is stopped"); - break; - } - - let mut tasks_buffer = cloned_tasks_buffer.lock().await; + usecase::executor::spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + loop { + if cancellation_token.is_cancelled() { + info!("Worker to clean tasks buffer is stopped"); + break; + } - if tasks_buffer.is_empty() { - drop(tasks_buffer); - tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; - continue; - } + let mut tasks_buffer = cloned_tasks_buffer.lock().await; - let end_point = { - if tasks_buffer.len() < tasks_batch_to_insert { - tasks_buffer.len() - } else { - tasks_batch_to_insert + if tasks_buffer.is_empty() { + drop(tasks_buffer); + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; + continue; } - }; - let mut tasks_to_insert = tasks_buffer.drain(0..end_point).collect::>(); + let end_point = { + if tasks_buffer.len() < tasks_batch_to_insert { + tasks_buffer.len() + } else { + tasks_batch_to_insert + } + }; - drop(tasks_buffer); + let mut tasks_to_insert = + tasks_buffer.drain(0..end_point).collect::>(); - let res = cloned_pg_client.insert_json_download_tasks(&mut tasks_to_insert).await; - match res { - Ok(_) => { - cloned_metrics.inc_tasks_set( - "tasks_set", - MetricStatus::SUCCESS, - tasks_to_insert.len() as u64, - ); - }, - Err(e) => { - cloned_metrics.inc_tasks_set( - "tasks_set", - MetricStatus::FAILURE, - tasks_to_insert.len() as u64, - ); - error!("insert_tasks: {}", e) - }, + drop(tasks_buffer); + + let res = + cloned_pg_client.insert_json_download_tasks(&mut tasks_to_insert).await; + match res { + Ok(_) => { + cloned_metrics.inc_tasks_set( + "tasks_set", + MetricStatus::SUCCESS, + tasks_to_insert.len() as u64, + ); + }, + Err(e) => { + cloned_metrics.inc_tasks_set( + "tasks_set", + MetricStatus::FAILURE, + tasks_to_insert.len() as u64, + ); + error!("insert_tasks: {}", e) + }, + } } } - - Ok(()) }); assets_iter.seek_to_first(); while assets_iter.valid() { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Setting tasks for JSONs is stopped"); break; } diff --git a/nft_ingester/src/bin/raw_backup/main.rs b/nft_ingester/src/bin/raw_backup/main.rs index 8e83446d5..772c70c3e 100644 --- a/nft_ingester/src/bin/raw_backup/main.rs +++ b/nft_ingester/src/bin/raw_backup/main.rs @@ -8,7 +8,6 @@ use rocks_db::{ column::TypedColumn, columns::offchain_data::OffChainData, migrator::MigrationState, Storage, }; use tempfile::TempDir; -use tokio::{sync::Mutex, task::JoinSet}; use tracing::info; #[derive(Parser, Debug)] @@ -28,25 +27,18 @@ pub async fn main() -> Result<(), IngesterError> { info!("Started..."); - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let secondary_rocks_dir = TempDir::new().unwrap(); let source_storage = Storage::open_secondary( config.source_db.as_str(), secondary_rocks_dir.path().to_str().unwrap(), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) .unwrap(); - let target_storage = Storage::open( - &config.target_db, - mutexed_tasks.clone(), - red_metrics.clone(), - MigrationState::Last, - ) - .unwrap(); + let target_storage = + Storage::open(&config.target_db, red_metrics.clone(), MigrationState::Last).unwrap(); let cf = &target_storage.db.cf_handle(OffChainData::NAME).unwrap(); info!("Copying offchain data..."); diff --git a/nft_ingester/src/bin/rocksdb_backup/main.rs b/nft_ingester/src/bin/rocksdb_backup/main.rs index 3f68ae127..f06c93ee1 100644 --- a/nft_ingester/src/bin/rocksdb_backup/main.rs +++ b/nft_ingester/src/bin/rocksdb_backup/main.rs @@ -9,7 +9,6 @@ use rocks_db::{ migrator::MigrationState, Storage, }; -use tokio::{sync::Mutex, task::JoinSet}; use tracing::{debug, info}; #[tokio::main(flavor = "multi_thread")] @@ -23,13 +22,9 @@ async fn main() -> Result<(), RocksDbBackupServiceError> { let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); red_metrics.register(&mut registry); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let storage = Storage::open_secondary( &args.rocks_db_path, &args.rocks_db_secondary_path, - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) diff --git a/nft_ingester/src/bin/slot_checker/main.rs b/nft_ingester/src/bin/slot_checker/main.rs index 0630329e7..8ea6436c7 100644 --- a/nft_ingester/src/bin/slot_checker/main.rs +++ b/nft_ingester/src/bin/slot_checker/main.rs @@ -18,10 +18,7 @@ use rocks_db::{ column::TypedColumn, columns::offchain_data::OffChainDataDeprecated, migrator::MigrationVersions, Storage, }; -use tokio::{ - signal, - sync::{broadcast, Mutex as AsyncMutex}, -}; +use tokio::{signal, sync::Mutex as AsyncMutex}; use tokio_util::sync::CancellationToken; use tracing::{error, info, warn}; use usecase::{bigtable::BigTableClient, slots_collector::SlotsCollector}; @@ -110,12 +107,12 @@ async fn main() -> Result<(), Box> { let args = Args::parse(); let metrics_state = MetricState::new(); + let cancellation_token = CancellationToken::new(); // Open target RocksDB in read-only mode let db = Arc::new( Storage::open_readonly_with_cfs( &args.target_db_path, vec![RawBlock::NAME, MigrationVersions::NAME, OffChainDataDeprecated::NAME], - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), metrics_state.red_metrics, ) .expect("Failed to open target RocksDB"), @@ -143,23 +140,17 @@ async fn main() -> Result<(), Box> { metrics_state.backfiller_metrics, ); - // Handle Ctrl+C - let shutdown_token = CancellationToken::new(); - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - - // Spawn a task to handle graceful shutdown on Ctrl+C - { - let shutdown_token = shutdown_token.clone(); - tokio::spawn(async move { + usecase::executor::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { if signal::ctrl_c().await.is_ok() { info!("Received Ctrl+C, shutting down gracefully..."); - shutdown_token.cancel(); - shutdown_tx.send(()).unwrap(); + cancellation_token.cancel(); } else { error!("Unable to listen for shutdown signal"); } - }); - } + } + }); // Check if slots or slots_file is provided let mut slots_to_check = Vec::new(); @@ -257,7 +248,7 @@ async fn main() -> Result<(), Box> { &BUBBLEGUM_PROGRAM_ID, last_persisted_slot, args.first_slot.unwrap_or_default(), - &shutdown_rx, + cancellation_token.child_token(), ) .await; @@ -268,7 +259,7 @@ async fn main() -> Result<(), Box> { let collected_slots = in_mem_dumper.get_sorted_keys().await; in_mem_dumper.clear().await; - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received, stopping..."); return Ok(()); } @@ -307,7 +298,7 @@ async fn main() -> Result<(), Box> { // Verification loop while let Some(slot) = next_slot { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received, stopping..."); break; } diff --git a/nft_ingester/src/bin/slot_persister/main.rs b/nft_ingester/src/bin/slot_persister/main.rs index c75a9b065..911e40086 100644 --- a/nft_ingester/src/bin/slot_persister/main.rs +++ b/nft_ingester/src/bin/slot_persister/main.rs @@ -14,10 +14,7 @@ use nft_ingester::{ inmemory_slots_dumper::InMemorySlotsDumper, }; use rocks_db::{column::TypedColumn, SlotStorage}; -use tokio::{ - sync::{broadcast, Semaphore}, - task::JoinSet, -}; +use tokio::{sync::Semaphore, task::JoinSet}; use tokio_retry::{strategy::ExponentialBackoff, RetryIf}; use tokio_util::sync::CancellationToken; use tracing::{debug, error, info, warn}; @@ -151,14 +148,17 @@ pub async fn main() -> Result<(), Box> { metrics_state.register_metrics(); start_metrics(metrics_state.registry, Some(args.metrics_port)).await; + let cancellation_token = CancellationToken::new(); + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + } + }); // Open target RocksDB let target_db = Arc::new( - SlotStorage::open( - &args.target_db_path, - Arc::new(tokio::sync::Mutex::new(tokio::task::JoinSet::new())), - metrics_state.red_metrics.clone(), - ) - .expect("Failed to open target RocksDB"), + SlotStorage::open(&args.target_db_path, metrics_state.red_metrics.clone()) + .expect("Failed to open target RocksDB"), ); let last_persisted_slot = get_last_persisted_slot(target_db.clone()); @@ -173,25 +173,6 @@ pub async fn main() -> Result<(), Box> { last_persisted_slot }; - let shutdown_token = CancellationToken::new(); - let shutdown_token_clone = shutdown_token.clone(); - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - - // Spawn a task to handle graceful shutdown on Ctrl+C - tokio::spawn(async move { - // Wait for Ctrl+C signal - match tokio::signal::ctrl_c().await { - Ok(()) => { - info!("Received Ctrl+C, shutting down gracefully..."); - shutdown_token_clone.cancel(); - shutdown_tx.send(()).unwrap(); - }, - Err(err) => { - error!("Unable to listen for shutdown signal: {}", err); - }, - } - }); - let rpc_client = Arc::new(BackfillRPC::connect(args.rpc_host.clone())); let backfill_source = { @@ -242,13 +223,19 @@ pub async fn main() -> Result<(), Box> { info!("Total slots to process: {}", provided_slots.len()); // Proceed to process the provided slots - process_slots(provided_slots, backfill_source, target_db, &args, shutdown_token.clone()) - .await; + process_slots( + provided_slots, + backfill_source, + target_db, + &args, + cancellation_token.child_token(), + ) + .await; return Ok(()); // Exit after processing provided slots } let mut start_slot = start_slot; loop { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received, exiting main loop..."); break; } @@ -265,7 +252,7 @@ pub async fn main() -> Result<(), Box> { &blockbuster::programs::bubblegum::ID, last_slot_to_check, start_slot, - &shutdown_rx, + cancellation_token.child_token(), ) .await; if let Some(slot) = top_collected_slot { @@ -286,7 +273,7 @@ pub async fn main() -> Result<(), Box> { tokio::select! { _ = sleep => {}, - _ = shutdown_token.cancelled() => { + _ = cancellation_token.cancelled() => { info!("Received shutdown signal, stopping loop..."); break; }, @@ -299,7 +286,7 @@ pub async fn main() -> Result<(), Box> { backfill_source.clone(), target_db.clone(), &args, - shutdown_token.clone(), + cancellation_token.child_token(), ) .await; }, @@ -311,12 +298,16 @@ pub async fn main() -> Result<(), Box> { let sleep = tokio::time::sleep(wait_period); tokio::select! { _ = sleep => {}, - _ = shutdown_token.cancelled() => { + _ = cancellation_token.cancelled() => { info!("Received shutdown signal, stopping loop..."); break; }, }; } + + if stop_handle.await.is_err() { + error!("Error joining graceful shutdown!"); + } info!("Slot persister has stopped."); Ok(()) } @@ -326,11 +317,11 @@ async fn process_slots( backfill_source: Arc, target_db: Arc, args: &Args, - shutdown_token: CancellationToken, + cancellation_token: CancellationToken, ) { // Process slots in batches for batch in slots.chunks(args.chunk_size) { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received during batch processing, exiting..."); break; } @@ -344,7 +335,7 @@ async fn process_slots( // Retry loop for the batch loop { - if shutdown_token.is_cancelled() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received during batch processing, exiting..."); break; } @@ -353,7 +344,7 @@ async fn process_slots( let backfill_source = backfill_source.clone(); let semaphore = Arc::new(Semaphore::new(args.max_concurrency)); - let shutdown_token = shutdown_token.clone(); + let shutdown_token = cancellation_token.clone(); match &*backfill_source { // ------------------------------------------------------------------ diff --git a/nft_ingester/src/bin/synchronizer/main.rs b/nft_ingester/src/bin/synchronizer/main.rs index acf1e8aff..69d4a9830 100644 --- a/nft_ingester/src/bin/synchronizer/main.rs +++ b/nft_ingester/src/bin/synchronizer/main.rs @@ -6,16 +6,13 @@ use metrics_utils::SynchronizerMetricsConfig; use nft_ingester::{ config::{init_logger, SynchronizerClapArgs}, error::IngesterError, - index_syncronizer::{SyncStatus, Synchronizer}, - init::{graceful_stop, init_index_storage_with_migration}, + index_synchronizer::{SyncStatus, Synchronizer}, + init::init_index_storage_with_migration, }; use postgre_client::PG_MIGRATIONS_PATH; use prometheus_client::registry::Registry; use rocks_db::{migrator::MigrationState, Storage}; -use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, -}; +use tokio::task::JoinSet; use tokio_util::sync::CancellationToken; #[cfg(feature = "profiling")] @@ -57,42 +54,34 @@ pub async fn main() -> Result<(), IngesterError> { .await?, ); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); + let cancellation_token = CancellationToken::new(); + let stop_handle = tokio::task::spawn({ + let cancellation_token = cancellation_token.clone(); + async move { + // --stop + #[cfg(not(feature = "profiling"))] + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + + #[cfg(feature = "profiling")] + nft_ingester::init::graceful_stop( + cancellation_token, + guard, + args.profiling_file_path_container, + &args.heap_path, + ) + .await; + } + }); let storage = Storage::open_secondary( &args.rocks_db_path, &args.rocks_db_secondary_path, - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) .unwrap(); let rocks_storage = Arc::new(storage); - let cloned_tasks = mutexed_tasks.clone(); - let (shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - let shutdown_token = CancellationToken::new(); - let shutdown_token_clone = shutdown_token.clone(); - - mutexed_tasks.lock().await.spawn(async move { - // --stop - #[cfg(not(feature = "profiling"))] - graceful_stop(cloned_tasks, shutdown_tx, Some(shutdown_token_clone)).await; - - #[cfg(feature = "profiling")] - graceful_stop( - cloned_tasks, - shutdown_tx, - Some(shutdown_token_clone), - guard, - args.profiling_file_path_container, - &args.heap_path, - ) - .await; - - Ok(()) - }); let synchronizer = Arc::new(Synchronizer::new( rocks_storage.clone(), @@ -110,46 +99,51 @@ pub async fn main() -> Result<(), IngesterError> { let mut sync_tasks = JoinSet::new(); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let shutdown_rx = shutdown_rx.resubscribe(); - let shutdown_token = shutdown_token.clone(); - sync_tasks.spawn(async move { - if let Ok(SyncStatus::FullSyncRequired(_)) = synchronizer - .get_sync_state(args.dump_sync_threshold, asset_type) - .await - { - tracing::info!("Starting full sync for {:?}", asset_type); - let res = synchronizer.full_syncronize(&shutdown_rx, asset_type).await; - match res { - Ok(_) => { - tracing::info!("Full {:?} synchronization finished successfully", asset_type); - } - Err(e) => { - tracing::error!("Full {:?} synchronization failed: {:?}", asset_type, e); + sync_tasks.spawn({ + let cancellation_token = cancellation_token.child_token(); + async move { + if cancellation_token.is_cancelled() { return; } + let sync_state_result = tokio::select! { + _ = cancellation_token.cancelled() => return, + sync_state_result = synchronizer + .get_sync_state(args.dump_sync_threshold, asset_type) => sync_state_result + + }; + if let Ok(SyncStatus::FullSyncRequired(_)) = sync_state_result + { + tracing::info!("Starting full sync for {:?}", asset_type); + let res = synchronizer.full_syncronize(cancellation_token.child_token(), asset_type).await; + match res { + Ok(_) => { + tracing::info!("Full {:?} synchronization finished successfully", asset_type); + } + Err(e) => { + tracing::error!("Full {:?} synchronization failed: {:?}", asset_type, e); + } } } - } - while shutdown_rx.is_empty() { - let result = synchronizer - .synchronize_asset_indexes(asset_type, &shutdown_rx, args.dump_sync_threshold) - .await; - - match result { - Ok(_) => { - tracing::info!("{:?} Synchronization finished successfully", asset_type) + while !cancellation_token.is_cancelled() { + let result = synchronizer + .synchronize_asset_indexes(asset_type, cancellation_token.child_token(), args.dump_sync_threshold) + .await; + + match result { + Ok(_) => { + tracing::info!("{:?} Synchronization finished successfully", asset_type) + } + Err(e) => tracing::error!("{:?} Synchronization failed: {:?}", asset_type, e), } - Err(e) => tracing::error!("{:?} Synchronization failed: {:?}", asset_type, e), - } - tokio::select! { - _ = tokio::time::sleep(tokio::time::Duration::from_secs( - args.timeout_between_syncs_sec, - )) => {} - _ = shutdown_token.cancelled() => { - tracing::info!("Shutdown signal received, stopping {:?} synchronizer", asset_type); - break; + tokio::select! { + _ = tokio::time::sleep(tokio::time::Duration::from_secs( + args.timeout_between_syncs_sec, + )) => {} + _ = cancellation_token.cancelled() => { + tracing::info!("Shutdown signal received, stopping {:?} synchronizer", asset_type); + break; + } } } } - }); } while let Some(task) = sync_tasks.join_next().await { @@ -158,7 +152,9 @@ pub async fn main() -> Result<(), IngesterError> { })?; } - while (mutexed_tasks.lock().await.join_next().await).is_some() {} + if stop_handle.await.is_err() { + tracing::error!("Error joining graceful shutdown!"); + } Ok(()) } diff --git a/nft_ingester/src/bin/synchronizer_utils/main.rs b/nft_ingester/src/bin/synchronizer_utils/main.rs index cf7c39eb4..93c9a91bf 100644 --- a/nft_ingester/src/bin/synchronizer_utils/main.rs +++ b/nft_ingester/src/bin/synchronizer_utils/main.rs @@ -13,7 +13,6 @@ use rocks_db::{ Storage, }; use solana_sdk::pubkey::Pubkey; -use tokio::{sync::Mutex, task::JoinSet}; use tracing::info; #[derive(Parser, Debug)] @@ -48,14 +47,11 @@ pub async fn main() -> Result<(), IngesterError> { let tx_storage_dir = tempfile::TempDir::new().unwrap(); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let red_metrics = Arc::new(metrics_utils::red::RequestErrorDurationMetrics::new()); let storage = Storage::open_secondary( &args.db_path, &tx_storage_dir.path().to_path_buf(), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) diff --git a/nft_ingester/src/cleaners/fork_cleaner.rs b/nft_ingester/src/cleaners/fork_cleaner.rs index ce243b4b4..05832b1bf 100644 --- a/nft_ingester/src/cleaners/fork_cleaner.rs +++ b/nft_ingester/src/cleaners/fork_cleaner.rs @@ -6,10 +6,10 @@ use metrics_utils::ForkCleanerMetricsConfig; use rocks_db::{SlotStorage, Storage}; use solana_sdk::{pubkey::Pubkey, signature::Signature}; use tokio::{ - sync::broadcast::Receiver, task::JoinError, time::{sleep as tokio_sleep, Instant}, }; +use tokio_util::sync::CancellationToken; use tracing::info; const CI_ITEMS_DELETE_BATCH_SIZE: usize = 100; @@ -18,18 +18,18 @@ const SLOT_CHECK_OFFSET: u64 = 1500; pub async fn run_fork_cleaner( fork_cleaner: ForkCleaner, metrics: Arc, - mut rx: Receiver<()>, + cancellation_token: CancellationToken, sequence_consistent_checker_wait_period_sec: u64, ) -> Result<(), JoinError> { info!("Start cleaning forks..."); loop { let start = Instant::now(); - fork_cleaner.clean_forks(rx.resubscribe()).await; + fork_cleaner.clean_forks(cancellation_token.child_token()).await; metrics.set_scans_latency(start.elapsed().as_secs_f64()); metrics.inc_total_scans(); tokio::select! { _ = tokio_sleep(Duration::from_secs(sequence_consistent_checker_wait_period_sec)) => {}, - _ = rx.recv() => { + _ = cancellation_token.cancelled() => { info!("Received stop signal, stopping cleaning forks!"); break; } @@ -62,10 +62,11 @@ where Self { cl_items_manager, fork_checker, metrics } } - pub async fn clean_forks(&self, rx: Receiver<()>) { + pub async fn clean_forks(&self, cancellation_token: CancellationToken) { let last_slot_for_check = self.fork_checker.last_slot_for_check().saturating_sub(SLOT_CHECK_OFFSET); - let all_non_forked_slots = self.fork_checker.get_all_non_forked_slots(rx.resubscribe()); + let all_non_forked_slots = + self.fork_checker.get_all_non_forked_slots(cancellation_token.child_token()); let mut forked_slots = 0; let mut delete_items = Vec::new(); @@ -73,7 +74,7 @@ where // from this column data will be dropped by slot // if we have any update from forked slot we have to delete it for cl_item in self.cl_items_manager.cl_items_iter() { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Stop iteration over cl items iterator..."); return; } diff --git a/nft_ingester/src/gapfiller.rs b/nft_ingester/src/gapfiller.rs index 40b0c4fa4..b1d28b524 100644 --- a/nft_ingester/src/gapfiller.rs +++ b/nft_ingester/src/gapfiller.rs @@ -10,15 +10,15 @@ use interface::{ use metrics_utils::{BackfillerMetricsConfig, SequenceConsistentGapfillMetricsConfig}; use rocks_db::Storage; use tokio::{ - sync::{broadcast::Receiver, Mutex}, - task::{JoinError, JoinSet}, + task::JoinError, time::{sleep as tokio_sleep, Instant}, }; +use tokio_util::sync::CancellationToken; use tracing::{error, log::info}; use usecase::slots_collector::SlotsGetter; pub async fn process_asset_details_stream_wrapper( - cloned_rx: Receiver<()>, + cancellation_token: CancellationToken, cloned_rocks_storage: Arc, last_saved_slot: u64, first_processed_slot_value: u64, @@ -27,7 +27,7 @@ pub async fn process_asset_details_stream_wrapper( ) -> Result<(), JoinError> { if raw_blocks { let processed_raw_blocks = process_raw_blocks_stream( - cloned_rx, + cancellation_token, cloned_rocks_storage, last_saved_slot, first_processed_slot_value, @@ -38,7 +38,7 @@ pub async fn process_asset_details_stream_wrapper( info!("Processed raw blocks: {}", processed_raw_blocks); } else { let processed_assets = process_asset_details_stream( - cloned_rx, + cancellation_token, cloned_rocks_storage.clone(), last_saved_slot, first_processed_slot_value, @@ -60,19 +60,17 @@ pub async fn run_sequence_consistent_gapfiller( sequence_consistent_gapfill_metrics: Arc, bp: Arc, bc: Arc, - rx: Receiver<()>, + cancellation_token: CancellationToken, rpc_backfiller: Arc, - mutexed_tasks: Arc>>>, sequence_consistent_checker_wait_period_sec: u64, ) where R: SlotsGetter + Sync + Send + 'static, BP: BlockProducer, BC: BlockConsumer, { - let mut rx = rx.resubscribe(); let metrics = sequence_consistent_gapfill_metrics.clone(); - let run_sequence_consistent_gapfiller = async move { + usecase::executor::spawn(async move { tracing::info!("Start collecting sequences gaps..."); loop { let start = Instant::now(); @@ -84,7 +82,7 @@ pub async fn run_sequence_consistent_gapfiller( sequence_consistent_gapfill_metrics.clone(), bp.clone(), bc.clone(), - rx.resubscribe(), + cancellation_token.child_token(), ) .await; metrics.set_scans_latency(start.elapsed().as_secs_f64()); @@ -92,23 +90,19 @@ pub async fn run_sequence_consistent_gapfiller( tokio::select! { _ = tokio_sleep(Duration::from_secs(sequence_consistent_checker_wait_period_sec)) => {}, - _ = rx.recv() => { + _ = cancellation_token.cancelled() => { info!("Received stop signal, stopping collecting sequences gaps"); break; } } } - - Ok(()) - }; - - mutexed_tasks.lock().await.spawn(run_sequence_consistent_gapfiller); + }); } /// Method returns the number of successfully processed assets #[allow(clippy::let_and_return)] pub async fn process_raw_blocks_stream( - _rx: Receiver<()>, + _cancellation_token: CancellationToken, _storage: Arc, _start_slot: u64, _end_slot: u64, @@ -154,7 +148,7 @@ pub async fn process_raw_blocks_stream( /// Method returns the number of successfully processed slots pub async fn process_asset_details_stream( - rx: Receiver<()>, + cancellation_token: CancellationToken, storage: Arc, start_slot: u64, end_slot: u64, @@ -173,7 +167,7 @@ pub async fn process_asset_details_stream( let mut processed_assets = 0; - while rx.is_empty() { + while !cancellation_token.is_cancelled() { match asset_details_stream.next().await { Some(Ok(details)) => { if let Some(e) = storage.insert_gaped_data(details).await.err() { diff --git a/nft_ingester/src/index_syncronizer.rs b/nft_ingester/src/index_synchronizer.rs similarity index 87% rename from nft_ingester/src/index_syncronizer.rs rename to nft_ingester/src/index_synchronizer.rs index 265b741d5..ca22cad78 100644 --- a/nft_ingester/src/index_syncronizer.rs +++ b/nft_ingester/src/index_synchronizer.rs @@ -10,6 +10,7 @@ use rocks_db::{ }; use solana_sdk::pubkey::Pubkey; use tokio::task::JoinSet; +use tokio_util::sync::CancellationToken; use tracing::warn; use crate::error::IngesterError; @@ -68,39 +69,37 @@ where pub async fn nft_run( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, run_full_sync_threshold: i64, - timeout_duration: tokio::time::Duration, ) { - while rx.is_empty() { - if let Err(e) = self.synchronize_nft_asset_indexes(rx, run_full_sync_threshold).await { + while !cancellation_token.is_cancelled() { + let cancellation_token = cancellation_token.child_token(); + if let Err(e) = self + .synchronize_nft_asset_indexes(cancellation_token, run_full_sync_threshold) + .await + { tracing::error!("Non fungible synchronization failed: {:?}", e); } else { tracing::info!("Non fungible synchronization finished successfully"); } - if rx.is_empty() { - tokio::time::sleep(timeout_duration).await; - } } } pub async fn fungible_run( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, run_full_sync_threshold: i64, - timeout_duration: tokio::time::Duration, ) { - while rx.is_empty() { - if let Err(e) = - self.synchronize_fungible_asset_indexes(rx, run_full_sync_threshold).await + while !cancellation_token.is_cancelled() { + let cancellation_token = cancellation_token.child_token(); + if let Err(e) = self + .synchronize_fungible_asset_indexes(cancellation_token, run_full_sync_threshold) + .await { tracing::error!("Fungible synchronization failed: {:?}", e); } else { tracing::info!("Fungible synchronization finished successfully"); } - if rx.is_empty() { - tokio::time::sleep(timeout_duration).await; - } } } @@ -172,22 +171,24 @@ where pub async fn synchronize_asset_indexes( &self, asset_type: AssetType, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, run_full_sync_threshold: i64, ) -> Result<(), IngesterError> { match asset_type { AssetType::NonFungible => { - self.synchronize_nft_asset_indexes(rx, run_full_sync_threshold).await + self.synchronize_nft_asset_indexes(cancellation_token, run_full_sync_threshold) + .await }, AssetType::Fungible => { - self.synchronize_fungible_asset_indexes(rx, run_full_sync_threshold).await + self.synchronize_fungible_asset_indexes(cancellation_token, run_full_sync_threshold) + .await }, } } pub async fn synchronize_nft_asset_indexes( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, run_full_sync_threshold: i64, ) -> Result<(), IngesterError> { let asset_type = AssetType::NonFungible; @@ -196,11 +197,21 @@ where match state { SyncStatus::FullSyncRequired(state) => { tracing::warn!("Should run dump synchronizer as the difference between last indexed and last known sequence is greater than the threshold. Last indexed: {:?}, Last known: {}", state.last_indexed_key.clone().map(|k|k.seq), state.last_known_key.seq); - self.regular_nft_syncronize(rx, state.last_indexed_key, state.last_known_key).await + self.regular_nft_syncronize( + cancellation_token, + state.last_indexed_key, + state.last_known_key, + ) + .await }, SyncStatus::RegularSyncRequired(state) => { tracing::debug!("Regular sync required for nft asset"); - self.regular_nft_syncronize(rx, state.last_indexed_key, state.last_known_key).await + self.regular_nft_syncronize( + cancellation_token, + state.last_indexed_key, + state.last_known_key, + ) + .await }, SyncStatus::NoSyncRequired => { tracing::debug!("No sync required for nft asset"); @@ -211,7 +222,7 @@ where pub async fn synchronize_fungible_asset_indexes( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, run_full_sync_threshold: i64, ) -> Result<(), IngesterError> { let asset_type = AssetType::Fungible; @@ -221,13 +232,21 @@ where match state { SyncStatus::FullSyncRequired(state) => { tracing::warn!("Should run dump synchronizer as the difference between last indexed and last known sequence is greater than the threshold. Last indexed: {:?}, Last known: {}", state.last_indexed_key.clone().map(|k|k.seq), state.last_known_key.seq); - self.regular_fungible_syncronize(rx, state.last_indexed_key, state.last_known_key) - .await + self.regular_fungible_syncronize( + cancellation_token, + state.last_indexed_key, + state.last_known_key, + ) + .await }, SyncStatus::RegularSyncRequired(state) => { tracing::debug!("Regular sync required for fungible asset"); - self.regular_fungible_syncronize(rx, state.last_indexed_key, state.last_known_key) - .await + self.regular_fungible_syncronize( + cancellation_token, + state.last_indexed_key, + state.last_known_key, + ) + .await }, SyncStatus::NoSyncRequired => { tracing::debug!("No sync required for fungible asset"); @@ -238,7 +257,7 @@ where pub async fn full_syncronize( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, asset_type: AssetType, ) -> Result<(), IngesterError> { let last_known_key = match asset_type { @@ -254,23 +273,28 @@ where }; let last_included_rocks_key = encode_u64x2_pubkey(last_known_key.seq, last_known_key.slot, last_known_key.pubkey); - self.dump_sync(last_included_rocks_key.as_slice(), rx, asset_type).await + self.dump_sync(last_included_rocks_key.as_slice(), cancellation_token, asset_type).await } async fn dump_sync( &self, last_included_rocks_key: &[u8], - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, asset_type: AssetType, ) -> Result<(), IngesterError> { tracing::info!("Dumping {:?} from the primary storage to {}", asset_type, self.dump_path); match asset_type { AssetType::NonFungible => { - self.dump_sync_nft(rx, last_included_rocks_key, NFT_SHARDS).await?; + self.dump_sync_nft(cancellation_token, last_included_rocks_key, NFT_SHARDS).await?; }, AssetType::Fungible => { - self.dump_sync_fungibles(rx, last_included_rocks_key, FUNGIBLE_SHARDS).await?; + self.dump_sync_fungibles( + cancellation_token, + last_included_rocks_key, + FUNGIBLE_SHARDS, + ) + .await?; }, } @@ -280,7 +304,7 @@ where async fn dump_sync_nft( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, last_included_rocks_key: &[u8], num_shards: u64, ) -> Result<(), IngesterError> { @@ -335,9 +359,9 @@ where let start = *start; let end = *end; - let shutdown_rx = rx.resubscribe(); let metrics = self.metrics.clone(); let rocks_storage = self.rocks_primary_storage.clone(); + let cancellation_token = cancellation_token.child_token(); tasks.spawn_blocking(move || { let res = rocks_storage.dump_nft_csv( assets_file, @@ -348,7 +372,7 @@ where None, Some(start), Some(end), - &shutdown_rx, + cancellation_token, metrics, )?; Ok(( @@ -393,7 +417,7 @@ where async fn dump_sync_fungibles( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, last_included_rocks_key: &[u8], num_shards: u64, ) -> Result<(), IngesterError> { @@ -422,9 +446,9 @@ where let start = *start; let end = *end; - let shutdown_rx = rx.resubscribe(); let metrics = self.metrics.clone(); let rocks_storage = self.rocks_primary_storage.clone(); + let cancellation_token = cancellation_token.child_token(); tasks.spawn_blocking(move || { let res = rocks_storage.dump_fungible_csv( @@ -432,7 +456,7 @@ where BUF_CAPACITY, Some(start), Some(end), - &shutdown_rx, + cancellation_token.child_token(), metrics, )?; Ok((res, fungible_tokens_path)) @@ -464,19 +488,19 @@ where async fn regular_fungible_syncronize( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, last_indexed_key: Option, last_key: AssetUpdatedKey, ) -> Result<(), IngesterError> { let mut starting_key = last_indexed_key; let mut processed_keys = HashSet::::new(); // Loop until no more new keys are returned - while rx.is_empty() { + while !cancellation_token.is_cancelled() { let mut tasks = JoinSet::new(); let mut last_included_rocks_key = None; let mut end_reached = false; for _ in 0..self.parallel_tasks { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { break; } let (updated_keys, last_included_key) = @@ -553,19 +577,19 @@ where async fn regular_nft_syncronize( &self, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, last_indexed_key: Option, last_key: AssetUpdatedKey, ) -> Result<(), IngesterError> { let mut starting_key = last_indexed_key; let mut processed_keys = HashSet::::new(); // Loop until no more new keys are returned - while rx.is_empty() { + while !cancellation_token.is_cancelled() { let mut tasks = JoinSet::new(); let mut last_included_rocks_key = None; let mut end_reached = false; for _ in 0..self.parallel_tasks { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { break; } let (updated_keys, last_included_key) = @@ -772,7 +796,7 @@ mod tests { } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_synchronizer_over_2_empty_storages() { let mut primary_storage = MockPrimaryStorage::new(); let mut index_storage = MockAssetIndexStorageMock::new(); @@ -795,29 +819,33 @@ mod tests { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); match asset_type { AssetType::Fungible => { - tokio::spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, AssetType::NonFungible => { - tokio::spawn(async move { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, } } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_synchronizer_with_records_in_primary_storage() { let mut primary_storage = MockPrimaryStorage::new(); let mut index_storage = MockAssetIndexStorageMock::new(); @@ -874,28 +902,32 @@ mod tests { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); match asset_type { AssetType::Fungible => { - tokio::spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, AssetType::NonFungible => { - tokio::spawn(async move { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, } } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_synchronizer_with_small_batch_size() { let mut primary_storage = MockPrimaryStorage::new(); let mut index_storage = MockAssetIndexStorageMock::new(); @@ -964,28 +996,32 @@ mod tests { metrics_state.synchronizer_metrics.clone(), 1, ); // Small batch size - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); match asset_type { AssetType::Fungible => { - tokio::spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, AssetType::NonFungible => { - tokio::spawn(async move { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, } } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_synchronizer_with_existing_index_data() { let mut primary_storage = MockPrimaryStorage::new(); let mut index_storage = MockAssetIndexStorageMock::new(); @@ -1097,28 +1133,32 @@ mod tests { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); match asset_type { AssetType::Fungible => { - tokio::spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, AssetType::NonFungible => { - tokio::spawn(async move { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, } } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_synchronizer_with_synced_databases() { let mut primary_storage = MockPrimaryStorage::new(); let mut index_storage = MockAssetIndexStorageMock::new(); @@ -1156,21 +1196,25 @@ mod tests { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); match asset_type { AssetType::Fungible => { - tokio::spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, AssetType::NonFungible => { - tokio::spawn(async move { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap(); + usecase::executor::spawn(async move { + synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(); }); }, } diff --git a/nft_ingester/src/init.rs b/nft_ingester/src/init.rs index 873abc54d..2ffb44493 100644 --- a/nft_ingester/src/init.rs +++ b/nft_ingester/src/init.rs @@ -1,6 +1,6 @@ #[cfg(feature = "profiling")] use std::{fs::File, io::Write}; -use std::{ops::DerefMut, path::PathBuf, sync::Arc}; +use std::{path::PathBuf, sync::Arc}; use metrics_utils::{red::RequestErrorDurationMetrics, MetricState}; use postgre_client::PgClient; @@ -10,11 +10,6 @@ use rocks_db::{migrator::MigrationState, Storage}; use tempfile::TempDir; #[cfg(feature = "profiling")] use tokio::process::Command; -use tokio::{ - sync::{broadcast::Sender, Mutex}, - task::{JoinError, JoinSet}, -}; -use tokio_util::sync::CancellationToken; #[cfg(feature = "profiling")] use tracing::error; @@ -53,11 +48,9 @@ pub async fn init_primary_storage( enable_migration_rocksdb: bool, migration_storage_path: &Option, metrics_state: &MetricState, - mutexed_tasks: Arc>>>, ) -> Result { Storage::open( db_path, - mutexed_tasks.clone(), metrics_state.red_metrics.clone(), MigrationState::CreateColumnFamilies, )?; @@ -67,7 +60,6 @@ pub async fn init_primary_storage( let migration_version_manager = Storage::open_secondary( db_path, migration_version_manager_dir.path().to_str().unwrap(), - mutexed_tasks.clone(), metrics_state.red_metrics.clone(), MigrationState::Last, )?; @@ -82,43 +74,17 @@ pub async fn init_primary_storage( .await?; } - Ok(Storage::open( - db_path, - mutexed_tasks.clone(), - metrics_state.red_metrics.clone(), - MigrationState::Last, - )?) -} - -#[cfg(not(feature = "profiling"))] -pub async fn graceful_stop( - tasks: Arc>>>, - shutdown_tx: Sender<()>, - shutdown_token: Option, -) { - usecase::graceful_stop::listen_shutdown().await; - let _ = shutdown_tx.send(()); - if let Some(token) = shutdown_token { - token.cancel(); - } - - usecase::graceful_stop::graceful_stop(tasks.lock().await.deref_mut()).await + Ok(Storage::open(db_path, metrics_state.red_metrics.clone(), MigrationState::Last)?) } #[cfg(feature = "profiling")] pub async fn graceful_stop( - tasks: Arc>>>, - shutdown_tx: Sender<()>, - shutdown_token: Option, + cancellation_token: tokio_util::sync::CancellationToken, guard: Option>, profile_path: Option, heap_path: &str, ) { - usecase::graceful_stop::listen_shutdown().await; - let _ = shutdown_tx.send(()); - if let Some(token) = shutdown_token { - token.cancel(); - } + usecase::graceful_stop::graceful_shutdown(cancellation_token).await; if let Some(guard) = guard { if let Ok(report) = guard.report().build() { @@ -133,8 +99,6 @@ pub async fn graceful_stop( if std::env::var(MALLOC_CONF_ENV).is_ok() { generate_profiling_gif(heap_path).await; } - - usecase::graceful_stop::graceful_stop(tasks.lock().await.deref_mut()).await } #[cfg(feature = "profiling")] diff --git a/nft_ingester/src/json_worker.rs b/nft_ingester/src/json_worker.rs index e749f48b1..a563d07b0 100644 --- a/nft_ingester/src/json_worker.rs +++ b/nft_ingester/src/json_worker.rs @@ -15,10 +15,11 @@ use rocks_db::{ }; use serde_json::Value; use tokio::{ - sync::{broadcast::Receiver, mpsc, mpsc::error::TryRecvError, Mutex}, + sync::{mpsc, mpsc::error::TryRecvError, Mutex}, task::JoinSet, time::{Duration, Instant}, }; +use tokio_util::sync::CancellationToken; use tracing::{debug, error}; use url::Url; @@ -74,9 +75,14 @@ impl TasksStreamer { Self { db_conn, sender, receiver } } - pub async fn run(self, rx: Receiver<()>, num_of_tasks: i32, tasks: &mut JoinSet<()>) { + pub fn run( + self, + cancellation_token: CancellationToken, + num_of_tasks: i32, + tasks: &mut JoinSet<()>, + ) { tasks.spawn(async move { - while rx.is_empty() { + while !cancellation_token.is_cancelled() { let locked_receiver = self.receiver.lock().await; let is_empty = locked_receiver.is_empty(); drop(locked_receiver); @@ -126,12 +132,12 @@ impl TasksPersister { Self { persister, receiver } } - pub async fn run(mut self, rx: Receiver<()>, tasks: &mut JoinSet<()>) { + pub fn run(mut self, cancellation_token: CancellationToken, tasks: &mut JoinSet<()>) { tasks.spawn(async move { let mut buffer = vec![]; let mut clock = tokio::time::Instant::now(); - while rx.is_empty() { + while !cancellation_token.is_cancelled() { if buffer.len() > JSON_BATCH || tokio::time::Instant::now() - clock > Duration::from_secs(WIPE_PERIOD_SEC) @@ -184,7 +190,7 @@ impl TasksPersister { } } -pub async fn run(json_downloader: Arc, rx: Receiver<()>) { +pub async fn run(json_downloader: Arc, cancellation_token: CancellationToken) { let mut workers_pool = JoinSet::new(); let num_of_tasks = json_downloader.num_of_parallel_workers; @@ -199,17 +205,17 @@ pub async fn run(json_downloader: Arc, rx: Receiver<()>) { let tasks_persister = TasksPersister::new(json_downloader.clone(), result_rx); - tasks_streamer.run(rx.resubscribe(), num_of_tasks, &mut workers_pool).await; - tasks_persister.run(rx.resubscribe(), &mut workers_pool).await; + tasks_streamer.run(cancellation_token.child_token(), num_of_tasks, &mut workers_pool); + tasks_persister.run(cancellation_token.child_token(), &mut workers_pool); for _ in 0..json_downloader.num_of_parallel_workers { - let cln_rx = rx.resubscribe(); let json_downloader = json_downloader.clone(); let tasks_rx = tasks_rx.clone(); let result_tx = result_tx.clone(); + let cancellation_token = cancellation_token.child_token(); workers_pool.spawn(async move { - while cln_rx.is_empty() { + while !cancellation_token.is_cancelled() { let mut locked_rx = tasks_rx.lock().await; match locked_rx.try_recv() { Ok(task) => { @@ -217,9 +223,10 @@ pub async fn run(json_downloader: Arc, rx: Receiver<()>) { let begin_processing = Instant::now(); - let response = json_downloader - .download_file(task.metadata_url.clone(), CLIENT_TIMEOUT) - .await; + let response = tokio::select! { + _ = cancellation_token.cancelled() => { break; }, + r = json_downloader.download_file(task.metadata_url.clone(), CLIENT_TIMEOUT) => r + }; json_downloader.metrics.set_latency_task_executed( "json_downloader", diff --git a/nft_ingester/src/lib.rs b/nft_ingester/src/lib.rs index b60d95cf3..99eb81942 100644 --- a/nft_ingester/src/lib.rs +++ b/nft_ingester/src/lib.rs @@ -9,7 +9,7 @@ pub mod consts; pub mod error; pub mod flatbuffer_mapper; pub mod gapfiller; -pub mod index_syncronizer; +pub mod index_synchronizer; pub mod init; pub mod inmemory_slots_dumper; pub mod inscription_raw_parsing; diff --git a/nft_ingester/src/processors/account_based/mpl_core_fee_indexing_processor.rs b/nft_ingester/src/processors/account_based/mpl_core_fee_indexing_processor.rs index f8b66f6f5..2b2145c10 100644 --- a/nft_ingester/src/processors/account_based/mpl_core_fee_indexing_processor.rs +++ b/nft_ingester/src/processors/account_based/mpl_core_fee_indexing_processor.rs @@ -9,11 +9,9 @@ use metrics_utils::IngesterMetricsConfig; use postgre_client::PgClient; use solana_client::nonblocking::rpc_client::RpcClient; use solana_program::{pubkey::Pubkey, rent::Rent, sysvar::rent}; -use tokio::{ - sync::{broadcast::Receiver, Mutex, RwLock}, - task::JoinSet, -}; -use tracing::{error, info}; +use tokio::sync::RwLock; +use tokio_util::sync::CancellationToken; +use tracing::error; use crate::error::IngesterError; @@ -26,7 +24,6 @@ pub struct MplCoreFeeProcessor { pub metrics: Arc, rpc_client: Arc, rent: Arc>, - join_set: Arc>>>, } impl MplCoreFeeProcessor { @@ -34,35 +31,31 @@ impl MplCoreFeeProcessor { storage: Arc, metrics: Arc, rpc_client: Arc, - join_set: Arc>>>, ) -> Result { let rent_account = rpc_client.get_account(&rent::ID).await?; let rent: Rent = bincode::deserialize(&rent_account.data)?; - Ok(Self { storage, metrics, rpc_client, rent: Arc::new(RwLock::new(rent)), join_set }) + Ok(Self { storage, metrics, rpc_client, rent: Arc::new(RwLock::new(rent)) }) } // on-chain programs can fetch rent without RPC call // but off-chain indexer need to make such calls in order // to get actual rent data - pub async fn update_rent(&self, mut rx: Receiver<()>) { + pub fn update_rent(&self, cancellation_token: CancellationToken) { let rpc_client = self.rpc_client.clone(); let rent = self.rent.clone(); - self.join_set.lock().await.spawn(tokio::spawn(async move { - while rx.is_empty() { + usecase::executor::spawn(async move { + loop { if let Err(e) = Self::fetch_actual_rent(rpc_client.clone(), rent.clone()).await { error!("fetch_actual_rent: {}", e); tokio::time::sleep(Duration::from_secs(5)).await; continue; } tokio::select! { - _ = rx.recv() => { - info!("Received stop signal, stopping update_rent..."); - return; - } - _ = tokio::time::sleep(FETCH_RENT_INTERVAL) => {}, + _ = cancellation_token.cancelled() => { break; } + _ = tokio::time::sleep(FETCH_RENT_INTERVAL) => {} } } - })); + }); } async fn fetch_actual_rent( diff --git a/nft_ingester/src/processors/accounts_processor.rs b/nft_ingester/src/processors/accounts_processor.rs index b3921fa38..522bf1b97 100644 --- a/nft_ingester/src/processors/accounts_processor.rs +++ b/nft_ingester/src/processors/accounts_processor.rs @@ -15,11 +15,8 @@ use postgre_client::PgClient; use rocks_db::{batch_savers::BatchSaveStorage, Storage}; use solana_client::nonblocking::rpc_client::RpcClient; use solana_program::pubkey::Pubkey; -use tokio::{ - sync::{broadcast::Receiver, Mutex}, - task::{JoinError, JoinSet}, - time::Instant, -}; +use tokio::time::Instant; +use tokio_util::sync::CancellationToken; use tracing::{debug, error}; use uuid::Uuid; @@ -58,9 +55,8 @@ lazy_static::lazy_static! { } #[allow(clippy::too_many_arguments)] -pub async fn run_accounts_processor( - rx: Receiver<()>, - mutexed_tasks: Arc>>>, +pub fn run_accounts_processor( + cancellation_token: CancellationToken, unprocessed_transactions_getter: Arc, rocks_storage: Arc, account_buffer_size: usize, @@ -69,36 +65,35 @@ pub async fn run_accounts_processor>, postgre_client: Arc, rpc_client: Arc, - join_set: Arc>>>, processor_name: Option, wellknown_fungible_accounts: HashMap, ) { - mutexed_tasks.lock().await.spawn(async move { + usecase::executor::spawn(async move { let account_processor = loop { match AccountsProcessor::build( - rx.resubscribe(), - fees_buffer_size, - unprocessed_transactions_getter.clone(), - metrics.clone(), - message_process_metrics.clone(), - postgre_client.clone(), - rpc_client.clone(), - join_set.clone(), - processor_name.clone(), - wellknown_fungible_accounts.clone() - ) - .await { + cancellation_token.child_token(), + fees_buffer_size, + unprocessed_transactions_getter.clone(), + metrics.clone(), + message_process_metrics.clone(), + postgre_client.clone(), + rpc_client.clone(), + processor_name.clone(), + wellknown_fungible_accounts.clone(), + ) + .await + { Ok(processor) => break processor, Err(e) => { error!(%e, "Failed to build accounts processor {:?}, retrying in {} seconds...", processor_name.clone(), ACCOUNT_PROCESSOR_RESTART_INTERVAL.as_secs()); tokio::time::sleep(ACCOUNT_PROCESSOR_RESTART_INTERVAL).await; - } + }, } }; - account_processor.process_accounts(rx, rocks_storage, account_buffer_size).await; - - Ok(()) + account_processor + .process_accounts(cancellation_token.child_token(), rocks_storage, account_buffer_size) + .await; }); } @@ -124,14 +119,13 @@ pub struct AccountsProcessor { #[allow(clippy::too_many_arguments)] impl AccountsProcessor { pub async fn build( - rx: Receiver<()>, + cancellation_token: CancellationToken, fees_batch_size: usize, unprocessed_account_getter: Arc, metrics: Arc, message_process_metrics: Option>, postgre_client: Arc, rpc_client: Arc, - join_set: Arc>>>, processor_name: Option, wellknown_fungible_accounts: HashMap, ) -> Result { @@ -140,9 +134,8 @@ impl AccountsProcessor { let mpl_core_processor = MplCoreProcessor::new(metrics.clone()); let inscription_processor = InscriptionsProcessor::new(metrics.clone()); let core_fees_processor = - MplCoreFeeProcessor::build(postgre_client, metrics.clone(), rpc_client, join_set) - .await?; - core_fees_processor.update_rent(rx).await; + MplCoreFeeProcessor::build(postgre_client, metrics.clone(), rpc_client).await?; + core_fees_processor.update_rent(cancellation_token.child_token()); Ok(Self { fees_batch_size, @@ -161,7 +154,7 @@ impl AccountsProcessor { pub async fn process_accounts( &self, - rx: Receiver<()>, + cancellation_token: CancellationToken, storage: Arc, accounts_batch_size: usize, ) { @@ -172,7 +165,7 @@ impl AccountsProcessor { let mut interval = tokio::time::interval(FLUSH_INTERVAL); let mut batch_fill_instant = Instant::now(); - while rx.is_empty() { + loop { tokio::select! { unprocessed_accounts = self.unprocessed_account_getter.next_accounts(accounts_batch_size) => { let unprocessed_accounts = match unprocessed_accounts { @@ -193,6 +186,7 @@ impl AccountsProcessor { self.flush(&mut batch_storage, &mut ack_ids, &mut interval, &mut batch_fill_instant); self.core_fees_processor.store_mpl_assets_fee(&std::mem::take(&mut core_fees)).await; } + _ = cancellation_token.cancelled() => { break; } } } self.flush(&mut batch_storage, &mut ack_ids, &mut interval, &mut batch_fill_instant); diff --git a/nft_ingester/src/processors/transaction_processor.rs b/nft_ingester/src/processors/transaction_processor.rs index 0184ed0c3..ef185f596 100644 --- a/nft_ingester/src/processors/transaction_processor.rs +++ b/nft_ingester/src/processors/transaction_processor.rs @@ -3,11 +3,7 @@ use std::{sync::Arc, time::Duration}; use chrono::Utc; use interface::signature_persistence::UnprocessedTransactionsGetter; use metrics_utils::MessageProcessMetricsConfig; -use tokio::{ - sync::{broadcast::Receiver, Mutex}, - task::{JoinError, JoinSet}, - time::sleep as tokio_sleep, -}; +use tokio_util::sync::CancellationToken; use tracing::error; use super::transaction_based::bubblegum_updates_processor::BubblegumTxProcessor; @@ -15,23 +11,24 @@ use crate::{error::IngesterError, redis_receiver::get_timestamp_from_id}; const TRANSACTIONS_GETTER_IDLE_TIMEOUT_MILLIS: u64 = 250; -pub async fn run_transaction_processor( - rx: Receiver<()>, - mutexed_tasks: Arc>>>, +pub fn run_transaction_processor( + cancellation_token: CancellationToken, unprocessed_transactions_getter: Arc, geyser_bubblegum_updates_processor: Arc, message_process_metrics: Option>, ) where TG: UnprocessedTransactionsGetter + Send + Sync + 'static, { - let run_transaction_processor = async move { - while rx.is_empty() { + usecase::executor::spawn(async move { + while !cancellation_token.is_cancelled() { let txs = match unprocessed_transactions_getter.next_transactions().await { Ok(txs) => txs, Err(err) => { error!("Get next transactions: {}", err); - tokio_sleep(Duration::from_millis(TRANSACTIONS_GETTER_IDLE_TIMEOUT_MILLIS)) - .await; + tokio::time::sleep(Duration::from_millis( + TRANSACTIONS_GETTER_IDLE_TIMEOUT_MILLIS, + )) + .await; continue; }, }; @@ -62,9 +59,5 @@ pub async fn run_transaction_processor( } } } - - Ok(()) - }; - - mutexed_tasks.lock().await.spawn(run_transaction_processor); + }); } diff --git a/nft_ingester/src/rocks_db.rs b/nft_ingester/src/rocks_db.rs index f3881ea2d..7f2eccf50 100644 --- a/nft_ingester/src/rocks_db.rs +++ b/nft_ingester/src/rocks_db.rs @@ -11,23 +11,19 @@ use std::{ use rocks_db::{ backup_service, errors::RocksDbBackupServiceError, storage_traits::AssetSlotStorage, Storage, }; -use tokio::{ - sync::broadcast::{Receiver, Sender}, - task::JoinError, - time::sleep as tokio_sleep, -}; +use tokio::{task::JoinError, time::sleep as tokio_sleep}; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::config::INGESTER_BACKUP_NAME; pub async fn receive_last_saved_slot( - cloned_rx: Receiver<()>, - cloned_tx: Sender<()>, + cancellation_token: CancellationToken, cloned_rocks_storage: Arc, first_processed_slot_clone: Arc, last_saved_slot: u64, ) -> Result<(), JoinError> { - while cloned_rx.is_empty() { + while !cancellation_token.is_cancelled() { match cloned_rocks_storage.last_saved_slot() { Ok(Some(slot)) if slot != last_saved_slot => { first_processed_slot_clone.store(slot, Ordering::Relaxed); @@ -35,7 +31,7 @@ pub async fn receive_last_saved_slot( }, Err(e) => { error!("Error while getting last saved slot: {}", e); - cloned_tx.send(()).ok(); + cancellation_token.cancel(); break; }, _ => {}, diff --git a/nft_ingester/src/scheduler.rs b/nft_ingester/src/scheduler.rs index 63d36e5d7..e76098401 100644 --- a/nft_ingester/src/scheduler.rs +++ b/nft_ingester/src/scheduler.rs @@ -15,6 +15,7 @@ use rocks_db::{ Storage, }; use solana_program::pubkey::Pubkey; +use tokio_util::sync::CancellationToken; use tracing::{info, log::error}; use crate::{api::dapi::rpc_asset_convertors::parse_files, error::IngesterError}; @@ -56,9 +57,12 @@ impl Scheduler { Scheduler { storage: rocks_storage, jobs } } - pub async fn run_in_background(mut scheduler: Scheduler) { - tokio::spawn(async move { - scheduler.run().await; + pub async fn run_in_background( + mut scheduler: Scheduler, + cancellation_token: CancellationToken, + ) { + usecase::executor::spawn(async move { + cancellation_token.run_until_cancelled(scheduler.run()).await; }); } diff --git a/nft_ingester/src/sequence_consistent.rs b/nft_ingester/src/sequence_consistent.rs index 5ffef47fa..5de5ca56f 100644 --- a/nft_ingester/src/sequence_consistent.rs +++ b/nft_ingester/src/sequence_consistent.rs @@ -7,7 +7,7 @@ use interface::{ slot_getter::FinalizedSlotGetter, }; use metrics_utils::{BackfillerMetricsConfig, SequenceConsistentGapfillMetricsConfig}; -use tokio::sync::broadcast::Receiver; +use tokio_util::sync::CancellationToken; use tracing::{info, warn}; use usecase::slots_collector::{SlotsCollector, SlotsGetter}; @@ -22,7 +22,7 @@ pub async fn collect_sequences_gaps( sequence_consistent_gapfill_metrics: Arc, bp: Arc, bc: Arc, - rx: Receiver<()>, + cancellation_token: CancellationToken, ) where R: SlotsGetter + Sync + Send + 'static, S: SequenceConsistentManager, @@ -33,7 +33,7 @@ pub async fn collect_sequences_gaps( let last_slot_to_look_for_gaps = finalized_slot_getter.get_finalized_slot_no_error().await; let mut prev_state = TreeState::default(); for current_state in sequence_consistent_manager.tree_sequence_iter() { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Stop iteration over tree iterator..."); return; } @@ -60,7 +60,12 @@ pub async fn collect_sequences_gaps( // fill the gap now, the dumper is the inmemory one, so we could fetch the slots and ingest all of those collector - .collect_slots(¤t_state.tree, current_state.slot, prev_state.slot, &rx) + .collect_slots( + ¤t_state.tree, + current_state.slot, + prev_state.slot, + cancellation_token.child_token(), + ) .await; let slots = in_memory_dumper.get_sorted_keys().await; for slot_num in slots { diff --git a/nft_ingester/tests/api_tests.rs b/nft_ingester/tests/api_tests.rs index c6eafdabd..cf46cdb0d 100644 --- a/nft_ingester/tests/api_tests.rs +++ b/nft_ingester/tests/api_tests.rs @@ -84,7 +84,7 @@ mod tests { use sqlx::QueryBuilder; use tempfile::TempDir; use testcontainers::clients::Cli; - use tokio::{sync::Mutex, task::JoinSet}; + use tokio_util::sync::CancellationToken; use usecase::proofs::MaybeProofChecker; const SLOT_UPDATED: u64 = 100; @@ -94,15 +94,13 @@ mod tests { 26, 235, 59, 85, 152, 160, 240, 0, 0, 0, 0, 1, ]); - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_search_assets() { let cnt = 20; let cli = Cli::default(); let (env, generated_assets) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let limit = 10; let before: Option; let after: Option; @@ -116,7 +114,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); assert!(res.is_object()); let res_obj: AssetList = serde_json::from_value(res).unwrap(); after = res_obj.cursor.clone(); @@ -147,7 +145,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); for i in 0..limit { assert_eq!( @@ -168,7 +166,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); for i in 0..limit { assert_eq!( @@ -189,7 +187,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert!(res_obj.items.is_empty(), "items should be empty"); } @@ -204,7 +202,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); for i in 0..limit { assert_eq!( @@ -225,7 +223,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert!(res_obj.items.is_empty(), "items should be empty"); } @@ -253,7 +251,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); for i in 0..limit { assert_eq!( @@ -275,7 +273,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -297,7 +295,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -319,7 +317,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); // calculate the number of assets with creator verified true let mut cnt = 0; @@ -342,7 +340,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -364,7 +362,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -386,7 +384,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -408,7 +406,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); assert_eq!(res_obj.items.len(), 1, "items length should be 1"); @@ -432,7 +430,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); @@ -446,15 +444,13 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_asset_none_grouping_with_token_standard() { let cnt = 20; let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let pb = Pubkey::new_unique(); let authority = Pubkey::new_unique(); @@ -557,7 +553,7 @@ mod tests { id: pb.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["grouping"], Value::Array(vec![])); assert_eq!(response["content"]["metadata"]["token_standard"], "NonFungible"); @@ -565,14 +561,12 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_metadata_sanitizer() { let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, 0, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let whitespace_options = " \t\n\r\x0B\x0C\u{00A0}\u{1680}\u{2000}\u{2001}\u{2002}\u{2003}\ \u{2004}\u{2005}\u{2006}\u{2007}\u{2008}\u{2009}\u{200A}\u{2028}\ @@ -680,7 +674,7 @@ mod tests { id: pb.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!( response["content"]["metadata"]["name"], format!("{} name {}", whitespace, whitespace) @@ -692,15 +686,13 @@ mod tests { } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_asset_without_offchain_data() { let cnt = 20; let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let pb = Pubkey::new_unique(); let authority = Pubkey::new_unique(); @@ -776,7 +768,7 @@ mod tests { id: pb.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["id"], pb.to_string()); assert_eq!(response["grouping"], Value::Array(vec![])); @@ -786,7 +778,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_fungible_asset() { let cnt = 20; @@ -794,8 +786,6 @@ mod tests { let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let token_updates_processor = TokenAccountsProcessor::new(Arc::new(IngesterMetricsConfig::new())); @@ -897,7 +887,7 @@ mod tests { id: mint_key.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["ownership"]["ownership_model"], "single"); assert_eq!(response["ownership"]["owner"], owner_key.to_string()); @@ -929,7 +919,7 @@ mod tests { id: mint_key.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["ownership"]["ownership_model"], "token"); assert_eq!(response["ownership"]["owner"], ""); @@ -938,7 +928,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_asset_programable_interface() { let cnt = 20; @@ -946,8 +936,6 @@ mod tests { let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let token_updates_processor = TokenAccountsProcessor::new(Arc::new(IngesterMetricsConfig::new())); @@ -1074,7 +1062,7 @@ mod tests { id: mint_accs[0].pubkey.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["id"], mint_accs[0].pubkey.to_string()); assert_eq!(response["interface"], "ProgrammableNFT".to_string()); @@ -1083,7 +1071,7 @@ mod tests { id: mint_accs[1].pubkey.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["id"], mint_accs[1].pubkey.to_string()); assert_eq!(response["interface"], "ProgrammableNFT".to_string()); @@ -1091,15 +1079,13 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_burnt() { let cnt = 20; let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let token_updates_processor = TokenAccountsProcessor::new(Arc::new(IngesterMetricsConfig::new())); @@ -1219,7 +1205,7 @@ mod tests { id: mint_key.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); assert_eq!(response["ownership"]["ownership_model"], "single"); assert_eq!(response["ownership"]["owner"], owner_key.to_string()); @@ -1229,7 +1215,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_asset_signatures() { let cnt = 20; let cli = Cli::default(); @@ -1421,7 +1407,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_token_accounts() { let cnt = 20; let cli = Cli::default(); @@ -1615,7 +1601,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_token_accounts_pagination() { let cnt = 20; let cli = Cli::default(); @@ -1839,15 +1825,13 @@ mod tests { assert_eq!(combined_10_30, first_30.token_accounts[10..]); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_assets_by_owner() { let cnt = 20; let cli = Cli::default(); let (env, generated_assets) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let ref_value = generated_assets.owners[8].clone(); let payload = GetAssetsByOwner { @@ -1863,7 +1847,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_assets_by_owner(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_assets_by_owner(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); @@ -1875,15 +1859,13 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_assets_by_group() { let cnt = 20; let cli = Cli::default(); let (env, generated_assets) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let ref_value = generated_assets.collections[12].clone(); let payload = GetAssetsByGroup { @@ -1900,7 +1882,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_assets_by_group(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_assets_by_group(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); @@ -1912,15 +1894,13 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_assets_by_creator() { let cnt = 20; let cli = Cli::default(); let (env, generated_assets) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let ref_value = generated_assets.dynamic_details[5].clone(); let payload = GetAssetsByCreator { @@ -1937,7 +1917,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_assets_by_creator(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_assets_by_creator(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); @@ -1949,7 +1929,7 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_assets_by_authority() { let cnt = 20; @@ -1957,8 +1937,6 @@ mod tests { let (env, generated_assets) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let ref_value = generated_assets.authorities[9].clone(); let payload = GetAssetsByAuthority { @@ -1974,7 +1952,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_assets_by_authority(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_assets_by_authority(payload).await.unwrap(); let res_obj: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res_obj.total, 1, "total should be 1"); @@ -2012,14 +1990,12 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_json_middleware() { let cnt = 20; let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, cnt, SLOT_UPDATED).await; - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let url = "http://someUrl.com".to_string(); @@ -2164,7 +2140,7 @@ mod tests { options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload).await.unwrap(); let expected_content: Value = serde_json::from_str( r#" @@ -2205,7 +2181,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_cannot_service_gaped_tree() { let cnt = 20; let cli = Cli::default(); @@ -2260,7 +2236,7 @@ mod tests { assert!(matches!(res, DasApiError::CannotServiceRequest)); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_core_account_fees() { let cnt = 20; let cli = Cli::default(); @@ -2307,15 +2283,13 @@ mod tests { assert_eq!(res.core_fees_account.len(), 0); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_search_assets_grand_total() { let total_assets = 2000; let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create(&cli, total_assets, SLOT_UPDATED).await; let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = SearchAssets { limit: Some(1000), page: Some(1), @@ -2326,7 +2300,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.grand_total, Some(total_assets as u32)); @@ -2340,7 +2314,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.grand_total, Some(0)); @@ -2353,7 +2327,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.grand_total, None); } @@ -2397,8 +2371,6 @@ mod tests { NATIVE_MINT_PUBKEY.to_string(), ); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = SearchAssets { limit: Some(1000), page: Some(1), @@ -2410,7 +2382,7 @@ mod tests { }, ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert!(res.native_balance.unwrap().total_price > 0.0); } @@ -2500,8 +2472,6 @@ mod tests { o.await.unwrap(); let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = GetAsset { id: generated_assets.collections.first().unwrap().pubkey.to_string(), options: Options { @@ -2510,7 +2480,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_asset(payload).await.unwrap(); let res: Asset = serde_json::from_value(res).unwrap(); assert_eq!( @@ -2588,7 +2558,7 @@ mod tests { id: generated_assets.collections.first().unwrap().pubkey.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let res = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_asset(payload).await.unwrap(); let res: Asset = serde_json::from_value(res).unwrap(); assert!(res.grouping.clone().unwrap().first().unwrap().collection_metadata.is_none()); @@ -2630,8 +2600,6 @@ mod tests { }).unwrap(); let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = GetAsset { id: asset_pk.to_string(), options: Options { @@ -2640,7 +2608,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_asset(payload).await.unwrap(); let res: Asset = serde_json::from_value(res).unwrap(); assert_eq!( res.inscription.unwrap().validation_hash.unwrap(), @@ -2656,7 +2624,7 @@ mod tests { ..Default::default() }, }; - let res = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_asset(payload).await.unwrap(); let res: Asset = serde_json::from_value(res).unwrap(); assert_eq!(res.inscription, None); assert_eq!(res.spl20, None); @@ -2664,9 +2632,6 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn test_search_assets_get_all_spec_classes() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let number_items = 100; let cli = Cli::default(); @@ -2704,7 +2669,7 @@ mod tests { token_type: Some(TokenType::All), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.items.len(), number_items); @@ -2712,9 +2677,6 @@ mod tests { #[tokio::test(flavor = "multi_thread")] async fn test_search_assets_freeze_delegate_authorities_for_fungible() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let number_items = 100; let cli = Cli::default(); @@ -2804,9 +2766,8 @@ mod tests { .transform_and_save_mint_account(&mut batch_storage, &mint, &Default::default()) .unwrap(); batch_storage.flush().unwrap(); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200_000, @@ -2815,28 +2776,19 @@ mod tests { 1, ); let synchronizer = Arc::new(synchronizer); - let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { - let rx = rx.resubscribe(); let synchronizer = synchronizer.clone(); - match asset_type { + let _ = match asset_type { AssetType::Fungible => { - tasks.spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await - }); + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await }, AssetType::NonFungible => { - tasks.spawn( - async move { synchronizer.synchronize_nft_asset_indexes(&rx, 0).await }, - ); + synchronizer.synchronize_nft_asset_indexes(CancellationToken::new(), 0).await }, - } - } - while let Some(res) = tasks.join_next().await { - if let Err(err) = res { - panic!("{err}"); - } + }; } let payload = SearchAssets { @@ -2853,7 +2805,7 @@ mod tests { }; let api = create_api(&env, None); - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.items.len(), 1); @@ -2873,7 +2825,7 @@ mod tests { let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create_noise(&cli, cnt, 100).await; - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200_000, @@ -3023,37 +2975,24 @@ mod tests { .transform_and_save_mint_account(&mut batch_storage, &mint2, &Default::default()) .unwrap(); batch_storage.flush().unwrap(); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); - let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { - let rx = rx.resubscribe(); let synchronizer = synchronizer.clone(); - match asset_type { + let _ = match asset_type { AssetType::Fungible => { - tasks.spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await - }); + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await }, AssetType::NonFungible => { - tasks.spawn( - async move { synchronizer.synchronize_nft_asset_indexes(&rx, 0).await }, - ); + synchronizer.synchronize_nft_asset_indexes(CancellationToken::new(), 0).await }, - } - } - - while let Some(res) = tasks.join_next().await { - if let Err(err) = res { - panic!("{err}"); - } + }; } let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = SearchAssets { limit: Some(1000), page: Some(1), @@ -3066,7 +3005,7 @@ mod tests { token_type: Some(TokenType::Fungible), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.items.len(), 2, "SearchAssets get by owner_address and token_type Fungible"); @@ -3111,7 +3050,7 @@ mod tests { token_type: Some(TokenType::All), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); // Totally we have 2 assets with required owner. show_fungible is false by default, so we don't have token info. @@ -3131,7 +3070,7 @@ mod tests { token_type: Some(TokenType::Fungible), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); // We have only 1 token account with non-zero balance @@ -3216,13 +3155,11 @@ mod tests { batch_storage.flush().unwrap(); let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = GetAsset { id: fungible_token_mint1.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let res = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.get_asset(payload).await.unwrap(); let res: Asset = serde_json::from_value(res).unwrap(); let reference = json!({ @@ -3269,7 +3206,7 @@ mod tests { ) .await; - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200_000, @@ -3358,37 +3295,23 @@ mod tests { .unwrap(); batch_storage.flush().unwrap(); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); - let synchronizer = Arc::new(synchronizer); - let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { - let rx = rx.resubscribe(); let synchronizer = synchronizer.clone(); - match asset_type { + let _ = match asset_type { AssetType::Fungible => { - tasks.spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await - }); + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await }, AssetType::NonFungible => { - tasks.spawn( - async move { synchronizer.synchronize_nft_asset_indexes(&rx, 0).await }, - ); + synchronizer.synchronize_nft_asset_indexes(CancellationToken::new(), 0).await }, - } - } - - while let Some(res) = tasks.join_next().await { - if let Err(err) = res { - panic!("{err}"); - } + }; } let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = SearchAssets { limit: Some(1000), page: Some(1), @@ -3403,7 +3326,7 @@ mod tests { token_type: Some(TokenType::Fungible), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); // We created 2 fungible tokens^ 1 with real pubkey (MPLX) @@ -3435,7 +3358,7 @@ mod tests { pg_mount.to_str().unwrap(), ) .await; - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200_000, @@ -3495,11 +3418,8 @@ mod tests { assert_eq!(idx_fungible_asset_iter.count(), 1); assert_eq!(idx_non_fungible_asset_iter.count(), cnt + 2); - let (_, rx) = tokio::sync::broadcast::channel(1); - for asset_type in ASSET_TYPES { - let rx = rx.resubscribe(); - synchronizer.full_syncronize(&rx, asset_type).await.expect("sync"); + synchronizer.full_syncronize(CancellationToken::new(), asset_type).await.expect("sync"); clean_syncronized_idxs( env.pg_env.client.clone(), env.rocks_env.storage.clone(), @@ -3601,7 +3521,7 @@ mod tests { temp_dir_path.to_str().unwrap(), ) .await; - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 10, @@ -3644,9 +3564,8 @@ mod tests { .unwrap(); batch_storage.flush().unwrap(); } - let (_tx, rx) = tokio::sync::broadcast::channel::<()>(1); for asset_type in ASSET_TYPES { - synchronizer.full_syncronize(&rx, asset_type).await.unwrap(); + synchronizer.full_syncronize(CancellationToken::new(), asset_type).await.unwrap(); } // receive 5 more updates for the same asset @@ -3724,7 +3643,7 @@ mod tests { ) } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_static_details_transition_from_fungible_into_nft() { // Given Fungible Metadata @@ -3756,8 +3675,6 @@ mod tests { Arc::new(RaydiumTokenPriceFetcher::default()), NATIVE_MINT_PUBKEY.to_string(), ); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let token_updates_processor = TokenAccountsProcessor::new(Arc::new(IngesterMetricsConfig::new())); @@ -3859,7 +3776,7 @@ mod tests { id: mint_key.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let response = api.get_asset(payload.clone(), mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload.clone()).await.unwrap(); assert_eq!(response["interface"], "FungibleToken".to_string()); @@ -3884,7 +3801,7 @@ mod tests { ) .unwrap(); batch_storage.flush().unwrap(); - let response = api.get_asset(payload.clone(), mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload.clone()).await.unwrap(); assert_eq!(response["interface"], "V1_NFT".to_string()); @@ -3908,7 +3825,7 @@ mod tests { ) .unwrap(); batch_storage.flush().unwrap(); - let response = api.get_asset(payload.clone(), mutexed_tasks.clone()).await.unwrap(); + let response = api.get_asset(payload.clone()).await.unwrap(); assert_eq!(response["interface"], "V1_NFT".to_string()); } @@ -3920,7 +3837,7 @@ mod tests { let cli = Cli::default(); let (env, _) = setup::TestEnvironment::create_noise(&cli, cnt, 100).await; - let synchronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200_000, @@ -4084,37 +4001,24 @@ mod tests { .transform_and_save_mint_account(&mut batch_storage, &mint2, &Default::default()) .unwrap(); batch_storage.flush().unwrap(); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let synchronizer = Arc::new(synchronizer); - let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { - let rx = rx.resubscribe(); let synchronizer = synchronizer.clone(); - match asset_type { + let _ = match asset_type { AssetType::Fungible => { - tasks.spawn(async move { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await - }); + synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await }, AssetType::NonFungible => { - tasks.spawn( - async move { synchronizer.synchronize_nft_asset_indexes(&rx, 0).await }, - ); + synchronizer.synchronize_nft_asset_indexes(CancellationToken::new(), 0).await }, - } - } - - while let Some(res) = tasks.join_next().await { - if let Err(err) = res { - panic!("{err}"); - } + }; } let api = create_api(&env, None); - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); let payload = SearchAssets { limit: Some(1000), @@ -4128,7 +4032,7 @@ mod tests { token_type: Some(TokenType::Fungible), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!( @@ -4149,7 +4053,7 @@ mod tests { token_type: Some(TokenType::All), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!(res.items.len(), 2, "SearchAssets by token_type:All show_zero_balance: true"); @@ -4168,7 +4072,7 @@ mod tests { token_type: Some(TokenType::NonFungible), ..Default::default() }; - let res = api.search_assets(payload, mutexed_tasks.clone()).await.unwrap(); + let res = api.search_assets(payload).await.unwrap(); let res: AssetList = serde_json::from_value(res).unwrap(); assert_eq!( diff --git a/nft_ingester/tests/asset_previews_tests.rs b/nft_ingester/tests/asset_previews_tests.rs index c0f262e0d..46df8a34d 100644 --- a/nft_ingester/tests/asset_previews_tests.rs +++ b/nft_ingester/tests/asset_previews_tests.rs @@ -10,7 +10,7 @@ mod tests { use setup::rocks::RocksTestEnvironment; use solana_sdk::keccak::{self, HASH_BYTES}; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_replace_file_urls_with_preview_urls() { let rocks_env = RocksTestEnvironment::new(&[]); @@ -124,7 +124,7 @@ mod tests { // This test demostrates that elements in the result of batch_get call // preserves same order as keys in argument - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_rocks_batch_get_order() { let rocks_env = RocksTestEnvironment::new(&[]); diff --git a/nft_ingester/tests/backfill_tests.rs b/nft_ingester/tests/backfill_tests.rs index ab64069b0..69cb59436 100644 --- a/nft_ingester/tests/backfill_tests.rs +++ b/nft_ingester/tests/backfill_tests.rs @@ -15,7 +15,7 @@ mod tests { use solana_program::pubkey::Pubkey; use usecase::{bigtable::BigTableClient, slots_collector::SlotsCollector}; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_consume_a_block_and_check_if_processed() { let (_tx, rx) = tokio::sync::broadcast::channel(1); @@ -45,7 +45,7 @@ mod tests { assert_eq!(persisted, original_block); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_collect_slots() { let (_tx, mut rx) = tokio::sync::broadcast::channel(1); @@ -66,7 +66,7 @@ mod tests { .await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_max_u64_request() { let (_tx, mut rx) = tokio::sync::broadcast::channel(1); diff --git a/nft_ingester/tests/batch_mint_test.rs b/nft_ingester/tests/batch_mint_test.rs index 7f8fbaab9..04964ff1d 100644 --- a/nft_ingester/tests/batch_mint_test.rs +++ b/nft_ingester/tests/batch_mint_test.rs @@ -57,7 +57,8 @@ use solana_transaction_status::{InnerInstruction, InnerInstructions, Transaction use spl_account_compression::ConcurrentMerkleTree; use tempfile::TempDir; use testcontainers::clients::Cli; -use tokio::{io::AsyncWriteExt, sync::broadcast}; +use tokio::io::AsyncWriteExt; +use tokio_util::sync::CancellationToken; use usecase::proofs::MaybeProofChecker; use uuid::Uuid; @@ -173,7 +174,7 @@ fn generate_merkle_tree_from_batch_mint(batch_mint: &BatchMint) -> ConcurrentMer merkle_tree } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn save_batch_mint_to_queue_test() { let cnt = 0; let cli = Cli::default(); @@ -269,7 +270,7 @@ async fn save_batch_mint_to_queue_test() { assert_eq!(r.url, metadata_url); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_with_verified_creators_test() { // For this test it's necessary to use Solana mainnet RPC let url = "https://api.mainnet-beta.solana.com".to_string(); @@ -368,8 +369,9 @@ async fn batch_mint_with_verified_creators_test() { let (batch_mint_to_verify, _) = env.rocks_env.storage.fetch_batch_mint_for_verifying().await.unwrap(); - let (_, rx) = broadcast::channel::<()>(1); - batch_mint_persister.persist_batch_mint(&rx, batch_mint_to_verify.unwrap(), None).await; + batch_mint_persister + .persist_batch_mint(CancellationToken::new(), batch_mint_to_verify.unwrap(), None) + .await; let api = nft_ingester::api::api_impl::DasApi::< MaybeProofChecker, @@ -401,7 +403,7 @@ async fn batch_mint_with_verified_creators_test() { assert_eq!(asset_proof.proof.is_empty(), false); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_with_unverified_creators_test() { // For this test it's necessary to use Solana mainnet RPC let url = "https://api.mainnet-beta.solana.com".to_string(); @@ -504,8 +506,9 @@ async fn batch_mint_with_unverified_creators_test() { let (batch_mint_to_verify, _) = env.rocks_env.storage.fetch_batch_mint_for_verifying().await.unwrap(); - let (_, rx) = broadcast::channel::<()>(1); - batch_mint_persister.persist_batch_mint(&rx, batch_mint_to_verify.unwrap(), None).await; + batch_mint_persister + .persist_batch_mint(CancellationToken::new(), batch_mint_to_verify.unwrap(), None) + .await; let api = nft_ingester::api::api_impl::DasApi::< MaybeProofChecker, @@ -543,7 +546,7 @@ async fn batch_mint_with_unverified_creators_test() { } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_persister_test() { let cnt = 0; let cli = Cli::default(); @@ -590,8 +593,9 @@ async fn batch_mint_persister_test() { let (batch_mint_to_verify, _) = env.rocks_env.storage.fetch_batch_mint_for_verifying().await.unwrap(); - let (_, rx) = broadcast::channel::<()>(1); - batch_mint_persister.persist_batch_mint(&rx, batch_mint_to_verify.unwrap(), None).await; + batch_mint_persister + .persist_batch_mint(CancellationToken::new(), batch_mint_to_verify.unwrap(), None) + .await; let merkle_tree = generate_merkle_tree_from_batch_mint(&test_batch_mint); @@ -681,7 +685,7 @@ async fn batch_mint_persister_test() { } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_persister_download_fail_test() { let cnt = 0; let cli = Cli::default(); @@ -729,15 +733,16 @@ async fn batch_mint_persister_download_fail_test() { let (batch_mint_to_verify, _) = env.rocks_env.storage.fetch_batch_mint_for_verifying().await.unwrap(); - let (_, rx) = broadcast::channel::<()>(1); - batch_mint_persister.persist_batch_mint(&rx, batch_mint_to_verify.unwrap(), None).await; + batch_mint_persister + .persist_batch_mint(CancellationToken::new(), batch_mint_to_verify.unwrap(), None) + .await; let r = env.rocks_env.storage.batch_mint_to_verify.get(metadata_hash.clone()).unwrap(); assert_eq!(r.is_none(), true); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_persister_drop_from_queue_after_download_fail_test() { let cnt = 0; let cli = Cli::default(); @@ -785,8 +790,9 @@ async fn batch_mint_persister_drop_from_queue_after_download_fail_test() { let (batch_mint_to_verify, _) = env.rocks_env.storage.fetch_batch_mint_for_verifying().await.unwrap(); - let (_, rx) = broadcast::channel::<()>(1); - batch_mint_persister.persist_batch_mint(&rx, batch_mint_to_verify.unwrap(), None).await; + batch_mint_persister + .persist_batch_mint(CancellationToken::new(), batch_mint_to_verify.unwrap(), None) + .await; let r = env.rocks_env.storage.batch_mint_to_verify.get(metadata_hash.clone()).unwrap(); @@ -802,7 +808,7 @@ async fn batch_mint_persister_drop_from_queue_after_download_fail_test() { assert_eq!(failed_batch_mint.download_attempts, download_attempts + 1); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn xxhash_test() { let file_data = vec![43, 2, 5, 4, 76, 34, 123, 42, 73, 81, 47]; @@ -826,7 +832,7 @@ async fn save_temp_batch_mint( file_name } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn batch_mint_upload_test() { let cnt = 0; let cli = Cli::default(); @@ -855,10 +861,9 @@ async fn batch_mint_upload_test() { Arc::new(BatchMintProcessorMetricsConfig::new()), ); - let (_, shutdown_rx) = broadcast::channel::<()>(1); let processing_result = batch_mint_processor .process_batch_mint( - shutdown_rx.resubscribe(), + CancellationToken::new(), BatchMintWithState { file_name, state: BatchMintState::Uploaded, @@ -890,7 +895,7 @@ async fn batch_mint_upload_test() { let file_name = save_temp_batch_mint(&dir, env.pg_env.client.clone(), &batch_mint).await; let processing_result = batch_mint_processor .process_batch_mint( - shutdown_rx.resubscribe(), + CancellationToken::new(), BatchMintWithState { file_name, state: BatchMintState::Uploaded, diff --git a/nft_ingester/tests/bubblegum_tests.rs b/nft_ingester/tests/bubblegum_tests.rs index 544c36f5d..25a23a7fd 100644 --- a/nft_ingester/tests/bubblegum_tests.rs +++ b/nft_ingester/tests/bubblegum_tests.rs @@ -22,10 +22,6 @@ mod tests { use rocks_db::{columns::offchain_data::OffChainData, migrator::MigrationState, Storage}; use solana_program::pubkey::Pubkey; use testcontainers::clients::Cli; - use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, - }; use usecase::proofs::MaybeProofChecker; // corresponds to So11111111111111111111111111111111111111112 @@ -34,7 +30,7 @@ mod tests { 26, 235, 59, 85, 152, 160, 240, 0, 0, 0, 0, 1, ]); - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] #[ignore = "FIXME: column families not opened error (probably outdated)"] async fn test_bubblegum_proofs() { @@ -45,9 +41,6 @@ mod tests { 242943774, 242947970, 242948187, 242949333, 242949940, 242951695, 242952638, ]; - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let tx_storage_dir = tempfile::TempDir::new().unwrap(); let storage_archieve = File::open("./tests/artifacts/test_rocks.zip").unwrap(); @@ -57,7 +50,6 @@ mod tests { let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let transactions_storage = Storage::open( &format!("{}{}", tx_storage_dir.path().to_str().unwrap(), "/test_rocks"), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) @@ -109,8 +101,6 @@ mod tests { )); let _producer = rocks_storage.clone(); - let (_shutdown_tx, _shutdown_rx) = broadcast::channel::<()>(1); - let file = File::open("./tests/artifacts/expected_proofs.json").unwrap(); let mut reader = io::BufReader::new(file); @@ -140,7 +130,7 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] #[ignore = "FIXME: column families not opened error (probably outdated)"] async fn test_asset_compression_info() { @@ -151,9 +141,6 @@ mod tests { 242943774, 242947970, 242948187, 242949333, 242949940, 242951695, 242952638, ]; - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let tx_storage_dir = tempfile::TempDir::new().unwrap(); let storage_archieve = File::open("./tests/artifacts/test_rocks.zip").unwrap(); @@ -163,7 +150,6 @@ mod tests { let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let transactions_storage = Storage::open( &format!("{}{}", tx_storage_dir.path().to_str().unwrap(), "/test_rocks"), - mutexed_tasks.clone(), red_metrics.clone(), MigrationState::Last, ) @@ -215,8 +201,6 @@ mod tests { )); let _producer = rocks_storage.clone(); - let (_shutdown_tx, _shutdown_rx) = broadcast::channel::<()>(1); - let metadata = OffChainData { url: Some("https://supersweetcollection.notarealurl/token.json".to_string()), metadata: Some("{\"msg\": \"hallo\"}".to_string()), @@ -265,7 +249,7 @@ mod tests { id: asset.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let asset_info = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let asset_info = api.get_asset(payload).await.unwrap(); assert_eq!(asset_info["compression"], expected_results[*asset]); } diff --git a/nft_ingester/tests/clean_forks_test.rs b/nft_ingester/tests/clean_forks_test.rs index c4fbf23a1..258c97fec 100644 --- a/nft_ingester/tests/clean_forks_test.rs +++ b/nft_ingester/tests/clean_forks_test.rs @@ -19,12 +19,12 @@ use rocks_db::{ use setup::rocks::RocksTestEnvironment; use solana_sdk::{pubkey::Pubkey, signature::Signature}; use spl_account_compression::{events::ChangeLogEventV1, state::PathNode}; -use tokio::sync::broadcast; +use tokio_util::sync::CancellationToken; #[cfg(test)] #[cfg(feature = "integration_tests")] #[tracing_test::traced_test] -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn test_clean_forks() { use std::{ collections::{HashMap, HashSet}, @@ -34,6 +34,7 @@ async fn test_clean_forks() { use entities::models::{RawBlockWithTransactions, UpdateVersion, Updated}; use metrics_utils::{utils::start_metrics, MetricsTrait}; use rocks_db::columns::{cl_items::ClItemKey, leaf_signatures::LeafSignature}; + use tokio_util::sync::CancellationToken; let RocksTestEnvironment { storage, slot_storage, .. } = RocksTestEnvironment::new(&[]); let first_tree_key = @@ -586,14 +587,12 @@ async fn test_clean_forks() { metrics_state.register_metrics(); start_metrics(metrics_state.registry, Some(4444)).await; - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - let rx = shutdown_rx.resubscribe(); let fork_cleaner = ForkCleaner::new( storage.clone(), slot_storage.clone(), metrics_state.fork_cleaner_metrics.clone(), ); - fork_cleaner.clean_forks(rx.resubscribe()).await; + fork_cleaner.clean_forks(CancellationToken::new()).await; let forked_first_key_first_item = storage.cl_items.get(ClItemKey::new(103, first_tree_key)).unwrap(); @@ -656,7 +655,7 @@ async fn test_clean_forks() { } #[tracing_test::traced_test] -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn test_process_forked_transaction() { let metrics_state = MetricState::new(); let RocksTestEnvironment { storage, slot_storage, .. } = RocksTestEnvironment::new(&[]); @@ -841,14 +840,12 @@ async fn test_process_forked_transaction() { ) .unwrap(); - let (_shutdown_tx, shutdown_rx) = broadcast::channel::<()>(1); - let fork_cleaner = ForkCleaner::new( storage.clone(), slot_storage.clone(), metrics_state.fork_cleaner_metrics.clone(), ); - fork_cleaner.clean_forks(shutdown_rx.resubscribe()).await; + fork_cleaner.clean_forks(CancellationToken::new()).await; for cl_item in storage.cl_items.iter_start() { let (_, value) = cl_item.unwrap(); diff --git a/nft_ingester/tests/decompress.rs b/nft_ingester/tests/decompress.rs index 48f107124..f5811cd74 100644 --- a/nft_ingester/tests/decompress.rs +++ b/nft_ingester/tests/decompress.rs @@ -40,10 +40,6 @@ mod tests { use rocks_db::{batch_savers::BatchSaveStorage, columns::offchain_data::OffChainData, Storage}; use solana_sdk::pubkey::Pubkey; use testcontainers::clients::Cli; - use tokio::{ - sync::{broadcast, Mutex}, - task::JoinSet, - }; use usecase::proofs::MaybeProofChecker; // corresponds to So11111111111111111111111111111111111111112 @@ -55,7 +51,6 @@ mod tests { // 242856151 slot when decompress happened async fn process_bubblegum_transactions( - mutexed_tasks: Arc>>>, env_rocks: Arc, _buffer: Arc, ) { @@ -91,7 +86,6 @@ mod tests { "SLOT_ASSET_IDX", "OFFCHAIN_DATA", ], - mutexed_tasks.clone(), red_metrics.clone(), ) .unwrap(); @@ -111,8 +105,6 @@ mod tests { Arc::new(BackfillerMetricsConfig::new()), )); let _producer = rocks_storage.clone(); - - let (_shutdown_tx, _shutdown_rx) = broadcast::channel::<()>(1); } async fn process_accounts( @@ -218,12 +210,9 @@ mod tests { .unwrap(); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_decompress_ideal_flow() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let cnt = 20; let cli = Cli::default(); let (env, _generated_assets) = setup::TestEnvironment::create(&cli, cnt, 100).await; @@ -268,12 +257,7 @@ mod tests { let mint = Pubkey::from_str("7DvMvi5iw8a4ESsd3bArGgduhvUgfD95iQmgucajgMPQ").unwrap(); - process_bubblegum_transactions( - mutexed_tasks.clone(), - env.rocks_env.storage.clone(), - buffer.clone(), - ) - .await; + process_bubblegum_transactions(env.rocks_env.storage.clone(), buffer.clone()).await; let mut batch_storage = BatchSaveStorage::new( env.rocks_env.storage.clone(), @@ -295,7 +279,7 @@ mod tests { id: mint.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let asset_info = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let asset_info = api.get_asset(payload).await.unwrap(); assert_eq!(asset_info["compression"], expected_results["compression"]); assert_eq!(asset_info["grouping"], expected_results["grouping"]); @@ -309,12 +293,9 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_decompress_first_mint_then_decompress_same_slot() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let cnt = 20; let cli = Cli::default(); let (env, _generated_assets) = setup::TestEnvironment::create(&cli, cnt, 100).await; @@ -367,12 +348,7 @@ mod tests { process_accounts(&mut batch_storage, 242856151, &mint).await; batch_storage.flush().unwrap(); - process_bubblegum_transactions( - mutexed_tasks.clone(), - env.rocks_env.storage.clone(), - buffer.clone(), - ) - .await; + process_bubblegum_transactions(env.rocks_env.storage.clone(), buffer.clone()).await; let file = File::open("./tests/artifacts/expected_decompress_result.json").unwrap(); let mut reader = io::BufReader::new(file); @@ -386,7 +362,7 @@ mod tests { id: mint.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let asset_info = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let asset_info = api.get_asset(payload).await.unwrap(); assert_eq!(asset_info["compression"], expected_results["compression"]); assert_eq!(asset_info["grouping"], expected_results["grouping"]); @@ -400,12 +376,9 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_decompress_first_mint_then_decompress_diff_slots() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let cnt = 20; let cli = Cli::default(); let (env, _generated_assets) = setup::TestEnvironment::create(&cli, cnt, 100).await; @@ -458,12 +431,7 @@ mod tests { process_accounts(&mut batch_storage, 252856151, &mint).await; batch_storage.flush().unwrap(); - process_bubblegum_transactions( - mutexed_tasks.clone(), - env.rocks_env.storage.clone(), - buffer.clone(), - ) - .await; + process_bubblegum_transactions(env.rocks_env.storage.clone(), buffer.clone()).await; let file = File::open("./tests/artifacts/expected_decompress_result.json").unwrap(); let mut reader = io::BufReader::new(file); @@ -477,7 +445,7 @@ mod tests { id: mint.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let asset_info = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let asset_info = api.get_asset(payload).await.unwrap(); assert_eq!(asset_info["compression"], expected_results["compression"]); assert_eq!(asset_info["grouping"], expected_results["grouping"]); @@ -491,12 +459,9 @@ mod tests { env.teardown().await; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_decompress_first_decompress_then_mint_diff_slots() { - let tasks = JoinSet::new(); - let mutexed_tasks = Arc::new(Mutex::new(tasks)); - let cnt = 20; let cli = Cli::default(); let (env, _generated_assets) = setup::TestEnvironment::create(&cli, cnt, 100).await; @@ -541,12 +506,7 @@ mod tests { let mint = Pubkey::from_str("7DvMvi5iw8a4ESsd3bArGgduhvUgfD95iQmgucajgMPQ").unwrap(); - process_bubblegum_transactions( - mutexed_tasks.clone(), - env.rocks_env.storage.clone(), - buffer.clone(), - ) - .await; + process_bubblegum_transactions(env.rocks_env.storage.clone(), buffer.clone()).await; let mut batch_storage = BatchSaveStorage::new( env.rocks_env.storage.clone(), @@ -568,7 +528,7 @@ mod tests { id: mint.to_string(), options: Options { show_unverified_collections: true, ..Default::default() }, }; - let asset_info = api.get_asset(payload, mutexed_tasks.clone()).await.unwrap(); + let asset_info = api.get_asset(payload).await.unwrap(); assert_eq!(asset_info["compression"], expected_results["compression"]); assert_eq!(asset_info["grouping"], expected_results["grouping"]); diff --git a/nft_ingester/tests/dump_tests.rs b/nft_ingester/tests/dump_tests.rs index 12081d608..42e3862b1 100644 --- a/nft_ingester/tests/dump_tests.rs +++ b/nft_ingester/tests/dump_tests.rs @@ -10,7 +10,7 @@ mod tests { }; use metrics_utils::{IngesterMetricsConfig, SynchronizerMetricsConfig}; use nft_ingester::{ - index_syncronizer::Synchronizer, + index_synchronizer::Synchronizer, processors::account_based::token_updates_processor::TokenAccountsProcessor, }; use postgre_client::{ @@ -22,6 +22,7 @@ mod tests { use solana_program::pubkey::Pubkey; use tempfile::TempDir; use testcontainers::clients::Cli; + use tokio_util::sync::CancellationToken; // corresponds to So11111111111111111111111111111111111111112 pub const NATIVE_MINT_PUBKEY: Pubkey = Pubkey::new_from_array([ @@ -61,7 +62,6 @@ mod tests { } batch_storage.flush().unwrap(); - let (_tx, rx) = tokio::sync::broadcast::channel::<()>(1); let temp_dir = TempDir::new_in("./tmp").expect("Failed to create a temporary directory"); let temp_dir_path = temp_dir.path(); @@ -69,7 +69,7 @@ mod tests { let cli: Cli = Cli::default(); let pg_env = setup::pg::TestEnvironment::new_with_mount(&cli, temp_dir_path).await; let client = pg_env.client.clone(); - let syncronizer = Arc::new(Synchronizer::new( + let synchronizer = Arc::new(Synchronizer::new( storage, client.clone(), 2000, @@ -78,7 +78,7 @@ mod tests { 1, )); for asset_type in ASSET_TYPES { - syncronizer.full_syncronize(&rx, asset_type).await.unwrap(); + synchronizer.full_syncronize(CancellationToken::new(), asset_type).await.unwrap(); } assert_eq!(pg_env.count_rows_in_metadata().await.unwrap(), 1); @@ -138,7 +138,6 @@ mod mtg_441_tests { use serde_json::Value; use setup::{rocks::RocksTestEnvironmentSetup, TestEnvironment}; use testcontainers::clients::Cli; - use tokio::{sync::Mutex, task::JoinSet}; use usecase::proofs::MaybeProofChecker; use crate::tests::NATIVE_MINT_PUBKEY; @@ -183,7 +182,7 @@ mod mtg_441_tests { serde_json::from_value::(json).expect("Cannot parse 'Asset'.") } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn authority_none_collection_authority_some() { let cli = Cli::default(); @@ -202,15 +201,11 @@ mod mtg_441_tests { let first_pubkey = generated_assets.static_details.first().expect("Cannot get first pubkey.").pubkey; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let api_res = get_das_api(&env) - .get_asset( - GetAsset { - id: first_pubkey.to_string(), - options: Options { show_unverified_collections: true, ..Default::default() }, - }, - mutexed_tasks, - ) + .get_asset(GetAsset { + id: first_pubkey.to_string(), + options: Options { show_unverified_collections: true, ..Default::default() }, + }) .await; assert!(api_res.is_ok()); @@ -219,7 +214,7 @@ mod mtg_441_tests { assert!(res.id.eq(&first_pubkey.to_string())); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn authority_some_collection_authority_none() { let cli = Cli::default(); @@ -238,15 +233,11 @@ mod mtg_441_tests { let first_pubkey = generated_assets.static_details.first().expect("Cannot get first pubkey.").pubkey; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let api_res = get_das_api(&env) - .get_asset( - GetAsset { - id: first_pubkey.to_string(), - options: Options { show_unverified_collections: true, ..Default::default() }, - }, - mutexed_tasks, - ) + .get_asset(GetAsset { + id: first_pubkey.to_string(), + options: Options { show_unverified_collections: true, ..Default::default() }, + }) .await; assert!(api_res.is_ok()); let api_res = api_res.expect("Cannot run api call."); @@ -254,7 +245,7 @@ mod mtg_441_tests { assert!(res.id.eq(&first_pubkey.to_string())); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn authority_some_collection_authority_some() { let cli = Cli::default(); @@ -273,15 +264,11 @@ mod mtg_441_tests { let first_pubkey = generated_assets.static_details.first().expect("Cannot get first pubkey.").pubkey; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let api_res = get_das_api(&env) - .get_asset( - GetAsset { - id: first_pubkey.to_string(), - options: Options { show_unverified_collections: true, ..Default::default() }, - }, - mutexed_tasks, - ) + .get_asset(GetAsset { + id: first_pubkey.to_string(), + options: Options { show_unverified_collections: true, ..Default::default() }, + }) .await; assert!(api_res.is_ok()); let api_res = api_res.expect("Cannot run api call."); @@ -289,7 +276,7 @@ mod mtg_441_tests { assert!(res.id.eq(&first_pubkey.to_string())); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn authority_none_collection_authority_none() { let cli = Cli::default(); @@ -308,15 +295,11 @@ mod mtg_441_tests { let first_pubkey = generated_assets.static_details.first().expect("Cannot get first pubkey.").pubkey; - let mutexed_tasks = Arc::new(Mutex::new(JoinSet::new())); let api_res = get_das_api(&env) - .get_asset( - GetAsset { - id: first_pubkey.to_string(), - options: Options { show_unverified_collections: true, ..Default::default() }, - }, - mutexed_tasks, - ) + .get_asset(GetAsset { + id: first_pubkey.to_string(), + options: Options { show_unverified_collections: true, ..Default::default() }, + }) .await; assert!(api_res.is_ok()); let api_res = api_res.expect("Cannot run api call."); diff --git a/nft_ingester/tests/gapfiller_tests.rs b/nft_ingester/tests/gapfiller_tests.rs index 122c86d8e..d483ff121 100644 --- a/nft_ingester/tests/gapfiller_tests.rs +++ b/nft_ingester/tests/gapfiller_tests.rs @@ -13,7 +13,7 @@ use rocks_db::{ }; use solana_sdk::pubkey::Pubkey; use tempfile::TempDir; -use tokio::{sync::Mutex, task::JoinSet}; +use tokio_util::sync::CancellationToken; fn create_test_complete_asset_details(pubkey: Pubkey) -> AssetCompleteDetailsGrpc { AssetCompleteDetailsGrpc { @@ -23,18 +23,13 @@ fn create_test_complete_asset_details(pubkey: Pubkey) -> AssetCompleteDetailsGrp } } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] async fn test_process_asset_details_stream() { let temp_dir = TempDir::new().expect("Failed to create a temporary directory"); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let storage = Arc::new( - Storage::open( - temp_dir.path().to_str().unwrap(), - Arc::new(Mutex::new(JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .expect("Failed to create a database"), + Storage::open(temp_dir.path().to_str().unwrap(), red_metrics.clone(), MigrationState::Last) + .expect("Failed to create a database"), ); let first_key = Pubkey::new_unique(); @@ -43,7 +38,6 @@ async fn test_process_asset_details_stream() { let details1 = create_test_complete_asset_details(first_key.clone()); let details2 = create_test_complete_asset_details(second_key.clone()); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); let mut mock = MockAssetDetailsConsumer::new(); mock.expect_get_asset_details_consumable_stream_in_range().returning(move |_, _| { Ok(Box::pin(stream::iter(vec![ @@ -52,7 +46,7 @@ async fn test_process_asset_details_stream() { Err(AsyncError::from("test error")), ]))) }); - process_asset_details_stream(rx, storage.clone(), 100, 200, mock).await; + process_asset_details_stream(CancellationToken::new(), storage.clone(), 100, 200, mock).await; let selected_data = storage .db .get_pinned_cf( @@ -78,18 +72,14 @@ async fn test_process_asset_details_stream() { assert_eq!(selected_data.dynamic_details.unwrap().supply, Some(Updated::new(1, None, 10))); } -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[ignore = "TODO: unignore when process_raw_blocks_stream is fixed"] async fn test_process_raw_blocks_stream() { let temp_dir = TempDir::new().expect("Failed to create a temporary directory"); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let storage = Arc::new( - SlotStorage::open( - temp_dir.path().to_str().unwrap(), - Arc::new(Mutex::new(JoinSet::new())), - red_metrics.clone(), - ) - .expect("Failed to create a database"), + SlotStorage::open(temp_dir.path().to_str().unwrap(), red_metrics.clone()) + .expect("Failed to create a database"), ); let slot = 153; let blockhash = "blockhash"; diff --git a/nft_ingester/tests/process_accounts.rs b/nft_ingester/tests/process_accounts.rs index aa0534c5d..248870779 100644 --- a/nft_ingester/tests/process_accounts.rs +++ b/nft_ingester/tests/process_accounts.rs @@ -76,7 +76,7 @@ mod tests { } } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn token_update_process() { let first_mint = Pubkey::new_unique(); let second_mint = Pubkey::new_unique(); @@ -189,7 +189,7 @@ mod tests { assert_eq!(second_dynamic_from_db.supply.unwrap().value, 1); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn mplx_update_process() { let first_mint = Pubkey::new_unique(); let second_mint = Pubkey::new_unique(); @@ -297,7 +297,7 @@ mod tests { }; } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn mpl_core_update_process() { let first_mpl_core = Pubkey::new_unique(); let first_owner = Pubkey::new_unique(); @@ -472,7 +472,7 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn inscription_process() { // real world accounts let inscription_account_data = general_purpose::STANDARD_NO_PAD.decode("ZAuXKuQmRbsAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAF00JG/taM5xDErn+0mQMBbbBdJuhYeh30FuRLrqWSbfBhAAAABhcHBsaWNhdGlvbi90ZXh0AeOkcaHjppsua2rgJHv2TUkEEClH4Y96jMvvKr1caFZzE7QEAAAAAABDAAAAAUAAAABmNTMyMGVmMjhkNTM3NWQ3YjFhNmFlNzBlYzQzZWRkMTE1ZmQxMmVhOTMzZTAxNjUzMDZhNzg4ZGNiZWVjYTMxAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA").unwrap() ; diff --git a/nft_ingester/tests/scheduler_tests.rs b/nft_ingester/tests/scheduler_tests.rs index c60513c54..7e5320e2b 100644 --- a/nft_ingester/tests/scheduler_tests.rs +++ b/nft_ingester/tests/scheduler_tests.rs @@ -2,6 +2,7 @@ use assertables::{assert_contains, assert_contains_as_result}; use nft_ingester::scheduler::Scheduler; use rocks_db::columns::{asset_previews::UrlToDownload, offchain_data::OffChainData}; use setup::{await_async_for, rocks::RocksTestEnvironment}; +use tokio_util::sync::CancellationToken; const NFT_1: (&str, &str) = ( "https://yra5lrhegorsgx7upcyi5trrfiktyczlq5g3jst3yvgaefab36vq.arweave.net/xEHVxOQzoyNf9Hiwjs4xKhU8CyuHTbTKe8VMAhQB36s", @@ -48,7 +49,7 @@ const NFT_2: (&str, &str) = ( "#, ); -#[tokio::test] +#[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_collect_urls_to_download() { // This test checks that the job that serves for initial population of @@ -69,7 +70,7 @@ async fn test_collect_urls_to_download() { }); let sut = Scheduler::new(rocks_env.storage.clone(), None); - Scheduler::run_in_background(sut).await; + Scheduler::run_in_background(sut, CancellationToken::new()).await; await_async_for!( rocks_env.storage.urls_to_download.get_from_start(10).len() == 2, diff --git a/nft_ingester/tests/sequence_consistent_tests.rs b/nft_ingester/tests/sequence_consistent_tests.rs index 5056d1b30..dfc32498f 100644 --- a/nft_ingester/tests/sequence_consistent_tests.rs +++ b/nft_ingester/tests/sequence_consistent_tests.rs @@ -5,7 +5,7 @@ mod tests { #[cfg(feature = "integration_tests")] #[tracing_test::traced_test] - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_range_delete() { let storage = RocksTestEnvironment::new(&[]).storage; let first_tree_key = solana_program::pubkey::Pubkey::from_str( @@ -56,7 +56,7 @@ mod tests { #[cfg(feature = "rpc_tests")] #[tracing_test::traced_test] - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_fill_gap() { use std::sync::Arc; diff --git a/rocks-db/Cargo.toml b/rocks-db/Cargo.toml index 6462e5193..d076bb758 100644 --- a/rocks-db/Cargo.toml +++ b/rocks-db/Cargo.toml @@ -46,6 +46,7 @@ num-traits = { workspace = true } # using a different version of flatbuffer compared to the rest of the project as this one is compatible with generator used for AssetCompleteDetails structures flatbuffers = { version="24.3.25", features = ["serialize"]} indicatif = { workspace = true } +tokio-util = { workspace = true } [dev-dependencies] rand = { workspace = true } diff --git a/rocks-db/src/bin/column_copier/main.rs b/rocks-db/src/bin/column_copier/main.rs index 9ae2d8ae7..b7db92961 100644 --- a/rocks-db/src/bin/column_copier/main.rs +++ b/rocks-db/src/bin/column_copier/main.rs @@ -11,7 +11,7 @@ use rocks_db::{ column::TypedColumn, columns::offchain_data::OffChainData, migrator::MigrationState, Storage, }; use tempfile::TempDir; -use tokio::{sync::Mutex, task::JoinSet}; +use tokio::task::JoinSet; const BATCH_SIZE: usize = 10_000; @@ -65,18 +65,12 @@ async fn copy_column_families( let source_db = Storage::open_secondary( source_path, secondary_source_path, - Arc::new(Mutex::new(JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .map_err(|e| e.to_string())?; - let destination_db = Storage::open( - destination_path, - Arc::new(Mutex::new(JoinSet::new())), red_metrics.clone(), MigrationState::Last, ) .map_err(|e| e.to_string())?; + let destination_db = Storage::open(destination_path, red_metrics.clone(), MigrationState::Last) + .map_err(|e| e.to_string())?; let mut set = JoinSet::new(); // Create a MultiProgress to manage multiple progress bars diff --git a/rocks-db/src/bin/fork_detector/main.rs b/rocks-db/src/bin/fork_detector/main.rs index 7e58e940c..e682f5c85 100644 --- a/rocks-db/src/bin/fork_detector/main.rs +++ b/rocks-db/src/bin/fork_detector/main.rs @@ -9,7 +9,6 @@ use rocks_db::{ SlotStorage, Storage, }; use solana_sdk::pubkey::Pubkey; -use tokio::{sync::Mutex, task::JoinSet}; const BATCH_TO_DROP: usize = 1000; @@ -42,10 +41,8 @@ async fn find_forks(source_path: &str) -> Result<(), String> { println!("Opening DB..."); - let js = Arc::new(Mutex::new(JoinSet::new())); - let source_db = - Storage::open(source_path, js.clone(), red_metrics.clone(), MigrationState::Last) - .map_err(|e| e.to_string())?; + let source_db = Storage::open(source_path, red_metrics.clone(), MigrationState::Last) + .map_err(|e| e.to_string())?; println!("Opened in {:?}", start.elapsed()); @@ -53,7 +50,6 @@ async fn find_forks(source_path: &str) -> Result<(), String> { SlotStorage::open_secondary( source_path, // FIXME: provide correct paths for slots storage source_path, - js.clone(), red_metrics.clone(), ) .expect("should open slots db"), diff --git a/rocks-db/src/bin/leaf_checker/main.rs b/rocks-db/src/bin/leaf_checker/main.rs index 45879402a..c9a48a560 100644 --- a/rocks-db/src/bin/leaf_checker/main.rs +++ b/rocks-db/src/bin/leaf_checker/main.rs @@ -8,7 +8,7 @@ use rocks_db::{ Storage, }; use solana_sdk::pubkey::Pubkey; -use tokio::{sync::Mutex, task::JoinSet}; +use tokio::task::JoinSet; pub const NUM_OF_THREADS: usize = 2500; @@ -28,15 +28,8 @@ pub async fn main() { println!("Opening DB..."); - let source_db = Arc::new( - Storage::open( - source_db_path, - Arc::new(Mutex::new(JoinSet::new())), - red_metrics.clone(), - MigrationState::Last, - ) - .unwrap(), - ); + let source_db = + Arc::new(Storage::open(source_db_path, red_metrics.clone(), MigrationState::Last).unwrap()); println!("Opened in {:?}", start.elapsed()); diff --git a/rocks-db/src/bin/migrate_slots_db/main.rs b/rocks-db/src/bin/migrate_slots_db/main.rs index c2349183c..6ffa52fb7 100644 --- a/rocks-db/src/bin/migrate_slots_db/main.rs +++ b/rocks-db/src/bin/migrate_slots_db/main.rs @@ -5,10 +5,7 @@ use entities::models::{RawBlock, RawBlockDeprecated}; use metrics_utils::red::RequestErrorDurationMetrics; use rocks_db::{column::TypedColumn, errors::StorageError, SlotStorage, Storage}; use rocksdb::DB; -use tokio::{ - sync::{Mutex, Semaphore}, - task::{JoinError, JoinSet}, -}; +use tokio::sync::{Mutex, Semaphore}; #[derive(Parser)] #[command(author, version, about, long_about = None)] @@ -45,7 +42,6 @@ fn put_batch_vec( #[tokio::main(flavor = "multi_thread")] async fn main() { let args = Args::parse(); - let tasks = Arc::new(Mutex::new(JoinSet::>::new())); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let cf_descriptors = Storage::cfs_to_column_families( SlotStorage::cf_names() @@ -61,7 +57,7 @@ async fn main() { ) .expect("open rocks slot storage to migrate"), ); - let slot_storage = SlotStorage::new(db, tasks, red_metrics); + let slot_storage = SlotStorage::new(db, red_metrics); eprintln!("Opened slots database in primary mode at {}", args.slots_db_path); let mut iter = slot_storage.db.raw_iterator_cf( &slot_storage.db.cf_handle(RawBlockDeprecated::NAME).expect("get raw blocks cf handle"), diff --git a/rocks-db/src/clients/asset_streaming_client.rs b/rocks-db/src/clients/asset_streaming_client.rs index b09c91465..fda1bb8db 100644 --- a/rocks-db/src/clients/asset_streaming_client.rs +++ b/rocks-db/src/clients/asset_streaming_client.rs @@ -39,7 +39,7 @@ impl AssetDetailsStreamer for Storage { let (tx, rx) = tokio::sync::mpsc::channel(32); let backend = self.slot_asset_idx.backend.clone(); let metrics = self.red_metrics.clone(); - self.join_set.lock().await.spawn(tokio::spawn(async move { + usecase::executor::spawn(async move { let _ = process_asset_details_range( backend, start_slot, @@ -48,7 +48,7 @@ impl AssetDetailsStreamer for Storage { tx.clone(), ) .await; - })); + }); Ok(Box::pin(ReceiverStream::new(rx)) as AssetDetailsStream) } diff --git a/rocks-db/src/clients/dump_client.rs b/rocks-db/src/clients/dump_client.rs index 2ee7f7a5f..5c2d5f122 100644 --- a/rocks-db/src/clients/dump_client.rs +++ b/rocks-db/src/clients/dump_client.rs @@ -18,6 +18,7 @@ use inflector::Inflector; use metrics_utils::SynchronizerMetricsConfig; use serde::{Serialize, Serializer}; use solana_sdk::pubkey::Pubkey; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::{ @@ -102,7 +103,7 @@ impl Dumper for Storage { asset_limit: Option, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: Arc, ) -> Result { let mut metadata_key_set = HashSet::new(); @@ -337,7 +338,7 @@ impl Dumper for Storage { synchronizer_metrics.inc_num_of_records_written("authority", 1); } } - if !rx.is_empty() { + if cancellation_token.is_cancelled() { return Err("dump cancelled".to_string()); } iter.next(); @@ -371,7 +372,7 @@ impl Dumper for Storage { buf_capacity: usize, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: Arc, ) -> Result { let column: Column = Self::column(self.db.clone(), self.red_metrics.clone()); @@ -393,7 +394,7 @@ impl Dumper for Storage { break; } } - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Shutdown signal received..."); return Ok(cnt); } diff --git a/rocks-db/src/clients/raw_blocks_streaming_client.rs b/rocks-db/src/clients/raw_blocks_streaming_client.rs index 4a83bf27a..32cbf9e08 100644 --- a/rocks-db/src/clients/raw_blocks_streaming_client.rs +++ b/rocks-db/src/clients/raw_blocks_streaming_client.rs @@ -21,7 +21,7 @@ impl RawBlocksStreamer for SlotStorage { let (tx, rx) = tokio::sync::mpsc::channel(32); let backend = self.db.clone(); let metrics = self.red_metrics.clone(); - self.join_set.lock().await.spawn(tokio::spawn(async move { + usecase::executor::spawn(async move { let _ = process_raw_blocks_range( backend, start_slot, @@ -30,7 +30,7 @@ impl RawBlocksStreamer for SlotStorage { tx.clone(), ) .await; - })); + }); Ok(Box::pin(ReceiverStream::new(rx)) as RawBlocksStream) } diff --git a/rocks-db/src/fork_cleaner.rs b/rocks-db/src/fork_cleaner.rs index 8b2f9441c..b61845679 100644 --- a/rocks-db/src/fork_cleaner.rs +++ b/rocks-db/src/fork_cleaner.rs @@ -5,7 +5,7 @@ use entities::models::{ClItem, ForkedItem, LeafSignatureAllData, RawBlock}; use interface::fork_cleaner::{CompressedTreeChangesManager, ForkChecker}; use rocksdb::IteratorMode; use solana_sdk::{pubkey::Pubkey, signature::Signature}; -use tokio::sync::broadcast::Receiver; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::{ @@ -103,7 +103,7 @@ impl CompressedTreeChangesManager for Storage { #[async_trait] impl ForkChecker for SlotStorage { - fn get_all_non_forked_slots(&self, rx: Receiver<()>) -> HashSet { + fn get_all_non_forked_slots(&self, cancellation_token: CancellationToken) -> HashSet { let start_time = chrono::Utc::now(); let mut all_keys = HashSet::new(); for (key, _) in self @@ -111,7 +111,7 @@ impl ForkChecker for SlotStorage { .full_iterator_cf(&self.db.cf_handle(RawBlock::NAME).unwrap(), IteratorMode::Start) .filter_map(Result::ok) { - if !rx.is_empty() { + if cancellation_token.is_cancelled() { info!("Stop iteration over raw_blocks_cbor iterator..."); return all_keys; } diff --git a/rocks-db/src/lib.rs b/rocks-db/src/lib.rs index b0205a8fd..0dd180ebe 100644 --- a/rocks-db/src/lib.rs +++ b/rocks-db/src/lib.rs @@ -54,7 +54,6 @@ use flatbuffers::{FlatBufferBuilder, WIPOffset}; use inflector::Inflector; use metrics_utils::red::RequestErrorDurationMetrics; use rocksdb::{ColumnFamilyDescriptor, Options, DB}; -use tokio::{sync::Mutex, task::JoinSet}; use crate::{ errors::StorageError, @@ -76,42 +75,32 @@ const MAX_WRITE_BUFFER_SIZE: u64 = 256 * 1024 * 1024; // 256MB pub struct SlotStorage { pub db: Arc, pub raw_blocks: Column, - join_set: Arc>>>, red_metrics: Arc, } impl SlotStorage { - pub fn new( - db: Arc, - join_set: Arc>>>, - red_metrics: Arc, - ) -> Self { + pub fn new(db: Arc, red_metrics: Arc) -> Self { let raw_blocks = Storage::column(db.clone(), red_metrics.clone()); - Self { db, raw_blocks, red_metrics, join_set } + Self { db, raw_blocks, red_metrics } } pub fn cf_names() -> Vec<&'static str> { vec![RawBlock::NAME, MigrationVersions::NAME, OffChainDataDeprecated::NAME] } - pub fn open

( - db_path: P, - join_set: Arc>>>, - red_metrics: Arc, - ) -> Result + pub fn open

(db_path: P, red_metrics: Arc) -> Result where P: AsRef, { let cf_descriptors = Storage::cfs_to_column_families(Self::cf_names()); let db = Arc::new(DB::open_cf_descriptors(&Storage::get_db_options(), db_path, cf_descriptors)?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn open_secondary

( primary_path: P, secondary_path: P, - join_set: Arc>>>, red_metrics: Arc, ) -> Result where @@ -124,11 +113,10 @@ impl SlotStorage { secondary_path, cf_descriptors, )?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn open_readonly

( db_path: P, - join_set: Arc>>>, red_metrics: Arc, ) -> Result where @@ -136,7 +124,7 @@ impl SlotStorage { { let db = Arc::new(Storage::open_readonly_with_cfs_only_db(db_path, Self::cf_names())?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } } @@ -189,16 +177,11 @@ pub struct Storage { pub spl_mints: Column, assets_update_last_seq: AtomicU64, fungible_assets_update_last_seq: AtomicU64, - join_set: Arc>>>, red_metrics: Arc, } impl Storage { - fn new( - db: Arc, - join_set: Arc>>>, - red_metrics: Arc, - ) -> Self { + fn new(db: Arc, red_metrics: Arc) -> Self { let asset_static_data = Self::column(db.clone(), red_metrics.clone()); let asset_dynamic_data = Self::column(db.clone(), red_metrics.clone()); let asset_dynamic_data_deprecated = Self::column(db.clone(), red_metrics.clone()); @@ -271,7 +254,6 @@ impl Storage { slot_asset_idx, assets_update_last_seq: AtomicU64::new(0), fungible_assets_update_last_seq: AtomicU64::new(0), - join_set, tree_seq_idx, trees_gaps, token_metadata_edition_cbor, @@ -298,7 +280,6 @@ impl Storage { pub fn open

( db_path: P, - join_set: Arc>>>, red_metrics: Arc, migration_state: MigrationState, ) -> Result @@ -308,13 +289,12 @@ impl Storage { let cf_descriptors = Self::create_cf_descriptors(&migration_state); let db = Arc::new(DB::open_cf_descriptors(&Self::get_db_options(), db_path, cf_descriptors)?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn open_secondary

( primary_path: P, secondary_path: P, - join_set: Arc>>>, red_metrics: Arc, migration_state: MigrationState, ) -> Result @@ -328,13 +308,12 @@ impl Storage { secondary_path, cf_descriptors, )?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn open_cfs

( db_path: P, c_names: Vec<&str>, - join_set: Arc>>>, red_metrics: Arc, ) -> Result where @@ -343,20 +322,19 @@ impl Storage { let cf_descriptors = Self::cfs_to_column_families(c_names); let db = Arc::new(DB::open_cf_descriptors(&Self::get_db_options(), db_path, cf_descriptors)?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn open_readonly_with_cfs

( db_path: P, c_names: Vec<&str>, - join_set: Arc>>>, red_metrics: Arc, ) -> Result where P: AsRef, { let db = Arc::new(Self::open_readonly_with_cfs_only_db(db_path, c_names)?); - Ok(Self::new(db, join_set, red_metrics)) + Ok(Self::new(db, red_metrics)) } pub fn cfs_to_column_families(cfs: Vec<&str>) -> Vec { diff --git a/rocks-db/src/migrator.rs b/rocks-db/src/migrator.rs index ce02836b9..0701afbbd 100644 --- a/rocks-db/src/migrator.rs +++ b/rocks-db/src/migrator.rs @@ -9,7 +9,6 @@ use interface::migration_version_manager::PrimaryStorageMigrationVersionManager; use metrics_utils::red::RequestErrorDurationMetrics; use rocksdb::{IteratorMode, DB}; use serde::{de::DeserializeOwned, Deserialize, Serialize}; -use tokio::{sync::Mutex, task::JoinSet}; use tracing::{error, info}; use crate::{ @@ -196,7 +195,6 @@ impl<'a> MigrationApplier<'a> { fn open_migration_storage(db_path: &str, version: u64) -> Result { Storage::open( db_path, - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(version), ) diff --git a/rocks-db/src/storage_traits.rs b/rocks-db/src/storage_traits.rs index d5745e1a2..5916783d3 100644 --- a/rocks-db/src/storage_traits.rs +++ b/rocks-db/src/storage_traits.rs @@ -4,6 +4,7 @@ use async_trait::async_trait; use entities::models::{AssetIndex, FungibleAssetIndex}; use mockall::automock; use solana_sdk::pubkey::Pubkey; +use tokio_util::sync::CancellationToken; pub use crate::Result; use crate::Storage; @@ -66,7 +67,7 @@ pub trait Dumper { asset_limit: Option, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: std::sync::Arc, ) -> core::result::Result; @@ -76,7 +77,7 @@ pub trait Dumper { buf_capacity: usize, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: std::sync::Arc, ) -> core::result::Result; } @@ -156,7 +157,7 @@ impl Dumper for MockAssetIndexStorage { asset_limit: Option, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: std::sync::Arc, ) -> core::result::Result { self.mock_dumper.dump_nft_csv( @@ -168,7 +169,7 @@ impl Dumper for MockAssetIndexStorage { asset_limit, start_pubkey, end_pubkey, - rx, + cancellation_token, synchronizer_metrics, ) } @@ -178,7 +179,7 @@ impl Dumper for MockAssetIndexStorage { buf_capacity: usize, start_pubkey: Option, end_pubkey: Option, - rx: &tokio::sync::broadcast::Receiver<()>, + cancellation_token: CancellationToken, synchronizer_metrics: std::sync::Arc, ) -> core::result::Result { self.mock_dumper.dump_fungible_csv( @@ -186,7 +187,7 @@ impl Dumper for MockAssetIndexStorage { buf_capacity, start_pubkey, end_pubkey, - rx, + cancellation_token, synchronizer_metrics, ) } diff --git a/rocks-db/tests/asset_streaming_client_tests.rs b/rocks-db/tests/asset_streaming_client_tests.rs index dc3b5e90d..9e73b4783 100644 --- a/rocks-db/tests/asset_streaming_client_tests.rs +++ b/rocks-db/tests/asset_streaming_client_tests.rs @@ -8,7 +8,7 @@ mod tests { use solana_sdk::pubkey::Pubkey; use tokio_stream::StreamExt; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_asset_details_stream_in_range_empty_db() { let storage = RocksTestEnvironment::new(&[]).storage; @@ -22,7 +22,7 @@ mod tests { assert!(stream.next().await.is_none()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_asset_details_stream_in_range_data_only_before_target() { let storage = RocksTestEnvironment::new(&[]).storage; let pk = Pubkey::new_unique(); @@ -38,7 +38,7 @@ mod tests { assert!(stream.next().await.is_none()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_asset_details_stream_in_range_data_only_after_target() { let storage = RocksTestEnvironment::new(&[]).storage; let pk = Pubkey::new_unique(); @@ -54,7 +54,7 @@ mod tests { assert!(stream.next().await.is_none()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_asset_details_stream_in_range_data_missing_data() { let storage = RocksTestEnvironment::new(&[]).storage; let pk = Pubkey::new_unique(); @@ -76,7 +76,7 @@ mod tests { assert!(stream.next().await.is_none()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_asset_details_stream_in_range_data() { let cnt = 1000; let env = RocksTestEnvironment::new(&[]); @@ -99,7 +99,7 @@ mod tests { assert_eq!(pk_set, pks.pubkeys.into_iter().collect::>()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_get_raw_blocks_stream_in_range_data() { let env = RocksTestEnvironment::new(&[]); let slot_storage = &env.slot_storage; diff --git a/rocks-db/tests/batch_client_integration_tests.rs b/rocks-db/tests/batch_client_integration_tests.rs index 617064d66..bcd41d8b8 100644 --- a/rocks-db/tests/batch_client_integration_tests.rs +++ b/rocks-db/tests/batch_client_integration_tests.rs @@ -12,7 +12,6 @@ mod tests { use setup::rocks::{RocksTestEnvironment, DEFAULT_PUBKEY_OF_ONES, PUBKEY_OF_TWOS}; use solana_sdk::pubkey::Pubkey; use tempfile::TempDir; - use tokio::{sync::Mutex, task::JoinSet}; #[test] fn test_process_asset_updates_batch_empty_db() { @@ -20,7 +19,6 @@ mod tests { let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let storage = Storage::open( temp_dir.path().to_str().unwrap(), - Arc::new(Mutex::new(JoinSet::new())), red_metrics.clone(), MigrationState::Last, ) diff --git a/rocks-db/tests/dump_tests.rs b/rocks-db/tests/dump_tests.rs index f78a52532..c617c87f9 100644 --- a/rocks-db/tests/dump_tests.rs +++ b/rocks-db/tests/dump_tests.rs @@ -4,6 +4,7 @@ use metrics_utils::SynchronizerMetricsConfig; use rocks_db::storage_traits::Dumper; use setup::rocks::*; use tempfile::TempDir; +use tokio_util::sync::CancellationToken; #[tokio::test(flavor = "multi_thread", worker_threads = 10)] #[tracing_test::traced_test] @@ -12,7 +13,6 @@ async fn test_scv_export_from_rocks() { let number_of_assets = 1000; let _generated_assets = env.generate_assets(number_of_assets, 25).await; let storage = env.storage; - let (_tx, rx) = tokio::sync::broadcast::channel::<()>(1); let temp_dir = TempDir::new().expect("Failed to create a temporary directory"); let assets_path = format!("{}/assets.csv", temp_dir.path().to_str().unwrap()); let creators_path = format!("{}/creators.csv", temp_dir.path().to_str().unwrap()); @@ -35,7 +35,7 @@ async fn test_scv_export_from_rocks() { Some(number_of_assets), None, None, - &rx, + CancellationToken::new(), Arc::new(SynchronizerMetricsConfig::new()), ) .unwrap(); @@ -46,7 +46,7 @@ async fn test_scv_export_from_rocks() { 155, None, None, - &rx, + CancellationToken::new(), Arc::new(SynchronizerMetricsConfig::new()), ) .unwrap(); diff --git a/rocks-db/tests/migration_tests.rs b/rocks-db/tests/migration_tests.rs index 70c88c4d8..0caf19174 100644 --- a/rocks-db/tests/migration_tests.rs +++ b/rocks-db/tests/migration_tests.rs @@ -15,9 +15,8 @@ mod tests { }; use solana_sdk::pubkey::Pubkey; use tempfile::TempDir; - use tokio::{sync::Mutex, task::JoinSet}; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_migration() { let dir = TempDir::new().unwrap(); let v1 = OffChainDataDeprecated { @@ -33,7 +32,6 @@ mod tests { let path = dir.path().to_str().unwrap(); let old_storage = Storage::open( path, - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(0), ) @@ -51,7 +49,6 @@ mod tests { let migration_version_manager = Storage::open_secondary( path, secondary_storage_dir.path().to_str().unwrap(), - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(4), ) @@ -68,7 +65,6 @@ mod tests { let new_storage = Storage::open( path, - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(4), ) @@ -117,7 +113,7 @@ mod tests { assert_eq!(migrated_v2.last_read_at, 0); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_clitems_v2_migration() { let dir = TempDir::new().unwrap(); let node_id = 32782; @@ -142,7 +138,6 @@ mod tests { let path = dir.path().to_str().unwrap(); let old_storage = Storage::open( path, - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(0), ) @@ -153,7 +148,6 @@ mod tests { let migration_version_manager = Storage::open_secondary( path, secondary_storage_dir.path().to_str().unwrap(), - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(4), ) @@ -170,7 +164,6 @@ mod tests { let new_storage = Storage::open( path, - Arc::new(Mutex::new(JoinSet::new())), Arc::new(RequestErrorDurationMetrics::new()), MigrationState::Version(4), ) diff --git a/rocks-db/tests/parameters_tests.rs b/rocks-db/tests/parameters_tests.rs index 22e3feccf..8916d56ab 100644 --- a/rocks-db/tests/parameters_tests.rs +++ b/rocks-db/tests/parameters_tests.rs @@ -4,7 +4,7 @@ mod tests { use rocks_db::columns::parameters::Parameter; use setup::rocks::*; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_raw_block_on_empty_db() { let storage = RocksTestEnvironment::new(&[]).storage; @@ -22,7 +22,7 @@ mod tests { assert!(response == Some(last_fetched_slot)); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_merge_top_seen_slot() { let storage = RocksTestEnvironment::new(&[]).storage; diff --git a/rocks-db/tests/raw_block_tests.rs b/rocks-db/tests/raw_block_tests.rs index cc6134b95..c3df67e0e 100644 --- a/rocks-db/tests/raw_block_tests.rs +++ b/rocks-db/tests/raw_block_tests.rs @@ -6,7 +6,7 @@ mod tests { const RAW_BLOCK_DEPRECATED_CF_NAME: &str = "RAW_BLOCK_CBOR_ENCODED"; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_raw_block_encoding() { // case 1: mock data let raw_block_deprecated = RawBlockDeprecated { @@ -51,7 +51,7 @@ mod tests { assert_eq!(RawBlock::decode(&encoded).unwrap(), raw_block); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_raw_block_migration() { let env1 = RocksTestEnvironment::new(&[]); let raw_block_deprecated = RawBlockDeprecated { diff --git a/rocks-db/tests/signature_client_tests.rs b/rocks-db/tests/signature_client_tests.rs index b0e961268..c6cc3c1f2 100644 --- a/rocks-db/tests/signature_client_tests.rs +++ b/rocks-db/tests/signature_client_tests.rs @@ -5,7 +5,7 @@ mod tests { use setup::rocks::*; use solana_sdk::{pubkey::Pubkey, signature::Signature}; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_first_persisted_signature_for_empty_db() { let storage = RocksTestEnvironment::new(&[]).storage; let program_id = Pubkey::new_unique(); @@ -16,7 +16,7 @@ mod tests { assert!(response.is_none()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_persist_and_get_first_for_different_keys() { let storage = RocksTestEnvironment::new(&[]).storage; let first_program_id = Pubkey::new_unique(); @@ -63,7 +63,7 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_drop_signatures_before_for_empty_db() { let storage = RocksTestEnvironment::new(&[]).storage; let program_id = Pubkey::new_unique(); @@ -72,7 +72,7 @@ mod tests { assert!(response.is_ok()); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_drop_signatures_before_on_generated_records_for_2_accounts() { let storage = RocksTestEnvironment::new(&[]).storage; let first_program_id = Pubkey::new_unique(); @@ -132,7 +132,7 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] async fn test_drop_signatures_before_in_the_same_slot_faking_signature() { let storage = RocksTestEnvironment::new(&[]).storage; let first_program_id = Pubkey::new_unique(); diff --git a/rocks-db/tests/urls_to_download_test.rs b/rocks-db/tests/urls_to_download_test.rs index 7fa3fa016..671c50805 100644 --- a/rocks-db/tests/urls_to_download_test.rs +++ b/rocks-db/tests/urls_to_download_test.rs @@ -5,7 +5,7 @@ mod tests { use setup::{await_async_for, rocks::*}; use solana_sdk::keccak; - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_get_urls_to_download() { // prepare @@ -37,7 +37,7 @@ mod tests { ); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_successful_download() { // prepare @@ -64,7 +64,7 @@ mod tests { assert_eq!(preview_rul, Some(AssetPreviews { size: 400, failed: None })) } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_submit_absent_url() { // prepare @@ -80,7 +80,7 @@ mod tests { storage.submit_download_results(vec![download_result]).unwrap(); } - #[tokio::test] + #[tokio::test(flavor = "multi_thread")] #[tracing_test::traced_test] async fn test_remove_url_after_max_failed_attempts_to_dwonload() { // prepare diff --git a/tests/setup/Cargo.toml b/tests/setup/Cargo.toml index 8351a0e32..8929c3688 100644 --- a/tests/setup/Cargo.toml +++ b/tests/setup/Cargo.toml @@ -15,6 +15,7 @@ solana-sdk = { workspace = true } tempfile = { workspace = true } rand = { workspace = true } tokio = { workspace = true } +tokio-util = { workspace = true } sqlx = { workspace = true } testcontainers = { workspace = true } testcontainers-modules = { workspace = true } diff --git a/tests/setup/src/lib.rs b/tests/setup/src/lib.rs index 80cfc11b3..1806ec6e3 100644 --- a/tests/setup/src/lib.rs +++ b/tests/setup/src/lib.rs @@ -10,6 +10,7 @@ use rocks_db::columns::asset::{AssetAuthority, AssetCollection, AssetDynamicDeta use solana_sdk::pubkey::Pubkey; use testcontainers::clients::Cli; use tokio::task::JoinSet; +use tokio_util::sync::CancellationToken; use crate::rocks::RocksTestEnvironmentSetup; @@ -115,7 +116,7 @@ impl<'a> TestEnvironment<'a> { let mut metrics_state = metrics_utils::MetricState::new(); metrics_state.register_metrics(); - let syncronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200000, @@ -123,21 +124,21 @@ impl<'a> TestEnvironment<'a> { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); - let synchronizer = Arc::new(syncronizer); + let synchronizer = Arc::new(synchronizer); let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); tasks.spawn(async move { match asset_type { - AssetType::NonFungible => { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap() - }, - AssetType::Fungible => { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap() - }, + AssetType::NonFungible => synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(), + AssetType::Fungible => synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(), } }); } @@ -182,7 +183,7 @@ impl<'a> TestEnvironment<'a> { let mut metrics_state = metrics_utils::MetricState::new(); metrics_state.register_metrics(); - let syncronizer = nft_ingester::index_syncronizer::Synchronizer::new( + let synchronizer = nft_ingester::index_synchronizer::Synchronizer::new( env.rocks_env.storage.clone(), env.pg_env.client.clone(), 200000, @@ -190,21 +191,21 @@ impl<'a> TestEnvironment<'a> { metrics_state.synchronizer_metrics.clone(), 1, ); - let (_, rx) = tokio::sync::broadcast::channel::<()>(1); - let synchronizer = Arc::new(syncronizer); + let synchronizer = Arc::new(synchronizer); let mut tasks = JoinSet::new(); for asset_type in ASSET_TYPES { let synchronizer = synchronizer.clone(); - let rx = rx.resubscribe(); tasks.spawn(async move { match asset_type { - AssetType::NonFungible => { - synchronizer.synchronize_nft_asset_indexes(&rx, 0).await.unwrap() - }, - AssetType::Fungible => { - synchronizer.synchronize_fungible_asset_indexes(&rx, 0).await.unwrap() - }, + AssetType::NonFungible => synchronizer + .synchronize_nft_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(), + AssetType::Fungible => synchronizer + .synchronize_fungible_asset_indexes(CancellationToken::new(), 0) + .await + .unwrap(), } }); } diff --git a/tests/setup/src/rocks.rs b/tests/setup/src/rocks.rs index 7ae982e8e..5b3a29bcb 100644 --- a/tests/setup/src/rocks.rs +++ b/tests/setup/src/rocks.rs @@ -19,7 +19,6 @@ use rocks_db::{ use solana_sdk::pubkey::Pubkey; use sqlx::types::chrono::Utc; use tempfile::TempDir; -use tokio::{sync::Mutex, task::JoinSet}; const DEFAULT_TEST_URL: &str = "http://example.com"; @@ -46,21 +45,16 @@ impl RocksTestEnvironment { pub fn new(keys: &[(u64, Pubkey)]) -> Self { let storage_temp_dir = TempDir::new().expect("Failed to create a temporary directory"); let slot_storage_temp_dir = TempDir::new().expect("Failed to create a temporary directory"); - let join_set = Arc::new(Mutex::new(JoinSet::new())); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let storage = Storage::open( storage_temp_dir.path().to_str().unwrap(), - join_set.clone(), red_metrics.clone(), MigrationState::Last, ) .expect("Failed to create storage database"); - let slot_storage = SlotStorage::open( - slot_storage_temp_dir.path().to_str().unwrap(), - join_set, - red_metrics.clone(), - ) - .expect("Failed to create slot storage database"); + let slot_storage = + SlotStorage::open(slot_storage_temp_dir.path().to_str().unwrap(), red_metrics.clone()) + .expect("Failed to create slot storage database"); for &(slot, ref pubkey) in keys { storage.asset_updated(slot, *pubkey).expect("Cannot update assets."); diff --git a/tests/txn_forwarder/src/utils.rs b/tests/txn_forwarder/src/utils.rs index 3775efe05..1624d97dc 100644 --- a/tests/txn_forwarder/src/utils.rs +++ b/tests/txn_forwarder/src/utils.rs @@ -24,7 +24,7 @@ pub fn find_sigs<'a>( ) -> Result<(JoinHandle>, UnboundedReceiver), String> { let mut last_sig = None; let (tx, rx) = mpsc::unbounded_channel::(); - let jh = tokio::spawn(async move { + let jh = usecase::executor::spawn(async move { loop { let before = last_sig; let sigs = client @@ -82,4 +82,4 @@ impl Stream for Siggrabbenheimer { { self.chan.poll_recv(cx) } -} \ No newline at end of file +} diff --git a/usecase/Cargo.toml b/usecase/Cargo.toml index 846ec9fbd..e5abe9dad 100644 --- a/usecase/Cargo.toml +++ b/usecase/Cargo.toml @@ -34,3 +34,4 @@ bubblegum-batch-sdk = { workspace = true } solana-storage-proto = { workspace = true } prost = "0.11.9" futures = { workspace = true } +tokio-util = { workspace = true } diff --git a/usecase/src/executor.rs b/usecase/src/executor.rs new file mode 100644 index 000000000..740a62e2d --- /dev/null +++ b/usecase/src/executor.rs @@ -0,0 +1,27 @@ +use std::{future::Future, sync::OnceLock}; + +use tokio::{ + sync::Mutex, + task::{AbortHandle, JoinSet}, +}; + +static EXECUTOR: Mutex>> = Mutex::const_new(OnceLock::new()); + +pub fn spawn + Send + 'static>(future: F) -> AbortHandle { + tokio::task::block_in_place(|| { + let mut executor_guard = EXECUTOR.blocking_lock(); + let _ = executor_guard.get_or_init(JoinSet::new); + let executor = + executor_guard.get_mut().expect("executor join set to be initialized upon access"); + executor.spawn(async move { + let _ = future.await; + }) + }) +} + +pub(crate) async fn shutdown() { + let mut executor_guard = EXECUTOR.lock().await; + if let Some(executor) = executor_guard.get_mut() { + while executor.join_next().await.is_some() {} + } +} diff --git a/usecase/src/graceful_stop.rs b/usecase/src/graceful_stop.rs index 34033cb13..6dab0f550 100644 --- a/usecase/src/graceful_stop.rs +++ b/usecase/src/graceful_stop.rs @@ -1,8 +1,6 @@ -use tokio::{ - signal, - task::{JoinError, JoinSet}, -}; -use tracing::{error, info}; +use tokio::signal; +use tokio_util::sync::CancellationToken; +use tracing::error; pub async fn listen_shutdown() { match signal::ctrl_c().await { @@ -13,20 +11,8 @@ pub async fn listen_shutdown() { } } -pub async fn graceful_stop(tasks: &mut JoinSet>) { - while let Some(task) = tasks.join_next().await { - match task { - Ok(_) => { - info!("One of the tasks was finished") - }, - Err(err) if err.is_panic() => { - let err = err.into_panic(); - error!("Task panic: {:?}", err); - }, - Err(err) => { - let err = err.to_string(); - error!("Task error: {}", err); - }, - } - } +pub async fn graceful_shutdown(cancellation_token: CancellationToken) { + listen_shutdown().await; + cancellation_token.cancel(); + crate::executor::shutdown().await; } diff --git a/usecase/src/lib.rs b/usecase/src/lib.rs index 255db42cf..b2b443692 100644 --- a/usecase/src/lib.rs +++ b/usecase/src/lib.rs @@ -1,5 +1,6 @@ pub mod asset_streamer; pub mod bigtable; +pub mod executor; pub mod graceful_stop; pub mod merkle_tree; pub mod proofs; diff --git a/usecase/src/slots_collector.rs b/usecase/src/slots_collector.rs index 29f1b41de..140dbb5a2 100644 --- a/usecase/src/slots_collector.rs +++ b/usecase/src/slots_collector.rs @@ -6,7 +6,7 @@ use metrics_utils::{BackfillerMetricsConfig, MetricStatus}; use mockall::automock; use solana_bigtable_connection::bigtable::BigTableConnection; use solana_sdk::clock::Slot; -use tokio::sync::broadcast::Receiver; +use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::bigtable::SECONDS_TO_RETRY_GET_DATA_FROM_BG; @@ -90,7 +90,7 @@ where collected_pubkey: &solana_program::pubkey::Pubkey, slot_start_from: u64, slot_parse_until: u64, - rx: &Receiver<()>, + cancellation_token: CancellationToken, ) -> Option { let mut start_at_slot = slot_start_from; info!( @@ -98,11 +98,7 @@ where collected_pubkey, start_at_slot, slot_parse_until ); let mut top_slot_collected = None; - loop { - if !rx.is_empty() { - info!("Received stop signal, returning"); - return None; - } + while !cancellation_token.is_cancelled() { let slots = self .row_keys_getter .get_slots_sorted_desc(collected_pubkey, start_at_slot, GET_SIGNATURES_LIMIT) From fee9538befd27382a4f049fa0118b3f082350d71 Mon Sep 17 00:00:00 2001 From: Andrii Date: Mon, 3 Mar 2025 17:42:07 +0100 Subject: [PATCH 12/21] feat: improve the deployment process (#437) * MTG-1307 improve the deployment process - added the ability to select docker imager when starting docker compose --- .env.example | 2 ++ docker-compose.yaml | 43 ++++++------------------------------------- 2 files changed, 8 insertions(+), 37 deletions(-) diff --git a/.env.example b/.env.example index 4ca2f12a1..137e2e737 100644 --- a/.env.example +++ b/.env.example @@ -1,3 +1,5 @@ +DOCKER_IMAGE_TAG=develop-latest + RPC_HOST='https://mainnet-aura.metaplex.com/{personal_rpc_key}' #Configurable app parts that could be enabled or disabled. (values in the default positions) diff --git a/docker-compose.yaml b/docker-compose.yaml index e03ecde48..2aa4c6ff8 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -1,5 +1,6 @@ services: ingester: + image: ghcr.io/metaplex-foundation/aura-ingester:${DOCKER_IMAGE_TAG} container_name: ingester restart: always env_file: @@ -22,19 +23,13 @@ services: # - ${PROFILING_FILE_PATH}:${PROFILING_FILE_PATH_CONTAINER}:rw depends_on: - db - build: - context: . - dockerfile: docker/app.Dockerfile - # Replace for profiling: - # dockerfile: docker/profiling/app.Dockerfile - args: - BINARY: ingester stop_grace_period: 10m logging: options: max-size: "2048m" das-api: + image: ghcr.io/metaplex-foundation/aura-api:${DOCKER_IMAGE_TAG} container_name: das-api restart: always env_file: @@ -49,17 +44,13 @@ services: # - ./heaps:/usr/src/app/heaps:rw depends_on: - db - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: api stop_grace_period: 2m logging: options: max-size: "2048m" synchronizer: + image: ghcr.io/metaplex-foundation/aura-synchronizer:${DOCKER_IMAGE_TAG} container_name: synchronizer restart: always env_file: @@ -74,11 +65,6 @@ services: # - ./heaps:/usr/src/app/heaps:rw depends_on: - db - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: synchronizer stop_grace_period: 5m logging: options: @@ -96,16 +82,12 @@ services: - ${INTEGRITY_VERIFICATION_SLOTS_COLLECT_PATH}:${INTEGRITY_VERIFICATION_SLOTS_COLLECT_PATH_CONTAINER}:rw depends_on: - db - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: integrity-verification logging: options: max-size: "2048m" backfill: + image: ghcr.io/metaplex-foundation/aura-backfill:${DOCKER_IMAGE_TAG} container_name: backfill restart: no env_file: @@ -118,16 +100,12 @@ services: - ${ROCKS_DB_PATH}:${ROCKS_DB_PATH}:rw - ${ROCKS_SLOTS_DB_PATH}:${ROCKS_SLOTS_DB_PATH}:ro stop_grace_period: 5m - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: backfill logging: options: max-size: "2048m" slot-persister: + image: ghcr.io/metaplex-foundation/aura-slot_persister:${DOCKER_IMAGE_TAG} container_name: slot-persister restart: always env_file: @@ -140,11 +118,6 @@ services: - ${ROCKS_SLOTS_DB_PATH}:${ROCKS_SLOTS_DB_PATH}:rw - ./creds.json:/usr/src/app/creds.json stop_grace_period: 5m - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: slot_persister logging: options: max-size: "2048m" @@ -168,6 +141,7 @@ services: max-size: "100m" rocksdb-backup: + image: ghcr.io/metaplex-foundation/aura-rocksdb_backup:${DOCKER_IMAGE_TAG} container_name: rocksdb-backup env_file: - .env @@ -177,11 +151,6 @@ services: - ${ROCKS_BACKUP_DIR}:${ROCKS_BACKUP_DIR}:rw - ${ROCKS_BACKUP_ARCHIVES_DIR}:${ROCKS_BACKUP_ARCHIVES_DIR}:rw - ${ROCKS_DB_SECONDARY_PATH}/backup:${ROCKS_DB_SECONDARY_PATH}:rw - build: - context: . - dockerfile: docker/app.Dockerfile - args: - BINARY: rocksdb_backup stop_grace_period: 2m logging: options: From bf1b55755d7c1a0fd67748eac5b214fb3f9f9a85 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Mon, 3 Mar 2025 18:09:31 +0000 Subject: [PATCH 13/21] chore: bump rust toolchain to 1.85 (#438) --- .github/workflows/rust.yml | 4 ++-- docker/base.Dockerfile | 2 +- rust-toolchain.toml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/rust.yml b/.github/workflows/rust.yml index b2c36763c..5d4dd2fb1 100644 --- a/.github/workflows/rust.yml +++ b/.github/workflows/rust.yml @@ -22,7 +22,7 @@ jobs: - name: Install Rust specific version uses: actions-rs/toolchain@v1 with: - toolchain: 1.84 + toolchain: 1.85 profile: minimal components: clippy, rustfmt override: true @@ -90,7 +90,7 @@ jobs: - name: Install Rust specific version uses: actions-rs/toolchain@v1 with: - toolchain: 1.84 + toolchain: 1.85 profile: minimal components: clippy, rustfmt override: true diff --git a/docker/base.Dockerfile b/docker/base.Dockerfile index ff8ab6047..5bdfc8d23 100644 --- a/docker/base.Dockerfile +++ b/docker/base.Dockerfile @@ -1,5 +1,5 @@ # Initial stage: install cargo-chef -FROM rust:1.84.0-bookworm AS chef +FROM rust:1.85.0-bookworm AS chef RUN apt update && apt install -y libclang-dev protobuf-compiler RUN cargo install cargo-chef WORKDIR /rust diff --git a/rust-toolchain.toml b/rust-toolchain.toml index efd9dc3db..c1bc0a694 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,2 +1,2 @@ [toolchain] -channel = "1.84.0" +channel = "1.85.0" From 1cbe9f797643b36ff44d5c7714abc8d008fcb398 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Tue, 4 Mar 2025 11:05:56 +0000 Subject: [PATCH 14/21] fix(chore): Fix github PR creation in release prepare [MTG-1360] (#439) * chore: Clean cliff.toml * fix(chore): Fix github PR creation in release prepare --- .github/workflows/release-prepare.yml | 3 +-- cliff.toml | 8 ++++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/.github/workflows/release-prepare.yml b/.github/workflows/release-prepare.yml index e93465ab5..0b000ccfd 100644 --- a/.github/workflows/release-prepare.yml +++ b/.github/workflows/release-prepare.yml @@ -107,8 +107,7 @@ jobs: --base main \ --head ${{ env.RELEASE_BRANCH }} \ --title "Release v${{ inputs.version }}" \ - --body-file .changelog_content \ - --draft false + --body-file .changelog_content echo "Pull Request created. Please review the PR, make any necessary adjustments, and merge when ready." diff --git a/cliff.toml b/cliff.toml index 01d29d6e0..de57c6982 100644 --- a/cliff.toml +++ b/cliff.toml @@ -45,7 +45,7 @@ postprocessors = [ # parse the commits based on https://www.conventionalcommits.org conventional_commits = true # filter out the commits that are not conventional -filter_unconventional = false +filter_unconventional = true # process each line of a commit as an individual commit split_commits = false # regex for parsing and grouping commits @@ -61,7 +61,7 @@ commit_parsers = [ { message = "^test(\\(.*\\))?:", group = "Testing" }, { message = "^chore\\(release\\):", skip = true }, { message = "^chore\\(deps\\)(\\(.*\\))?:", group = "Dependencies" }, - { message = "^chore(\\(.*\\))?:", group = "Miscellaneous Tasks" }, + { message = "^chore(\\(.*\\))?:", group = "Build Process & Auxiliary Tools" }, { message = "^ci(\\(.*\\))?:", group = "CI/CD" }, # Simple commits without scope @@ -75,10 +75,10 @@ commit_parsers = [ { message = "^MTG-\\d+\\s+.*(feat|feature)", group = "Features" }, { message = "^MTG-\\d+\\s+.*(doc|docs)", group = "Documentation" }, { message = "^MTG-\\d+\\s+.*(test)", group = "Testing" }, - { message = "^MTG-\\d+\\s+", group = "Tasks" }, + { message = "^MTG-\\d+\\s+", group = "Miscellaneous Tasks" }, ] # filter out the commits that are not matched by commit parsers -filter_commits = false +filter_commits = true # glob pattern for matching git tags tag_pattern = "v[0-9]*.[0-9]*.[0-9]*" # regex for skipping tags From 4ae77f2b7a2fa0aef32396b112e9a1be45da14e6 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Wed, 5 Mar 2025 11:44:54 +0000 Subject: [PATCH 15/21] chore: merge release v0.5 (#442) * chore(ci): added missing cliff config for 0.5 release * chore: dropped the version of nft_ingester to the one before 0.5 * chore: bump rust toolchain to 1.85 (#438) * chore: cherry pick scripts for release * chore: Clean cliff.toml * chore: bump version to 0.5.0 * docs: add changelog for v0.5.0 * chore: bump version to 0.5.1-dev [skip ci] --------- Co-authored-by: GitHub Actions --- CHANGELOG.md | 146 ++++++++++++++++++++++++++++++ Cargo.lock | 26 +++--- backfill_rpc/Cargo.toml | 2 +- consistency_check/Cargo.toml | 2 +- entities/Cargo.toml | 2 +- grpc/Cargo.toml | 2 +- integration_tests/Cargo.toml | 2 +- integrity_verification/Cargo.toml | 2 +- interface/Cargo.toml | 2 +- metrics_utils/Cargo.toml | 4 +- nft_ingester/Cargo.toml | 2 +- postgre-client/Cargo.toml | 2 +- rocks-db/Cargo.toml | 2 +- tests/setup/Cargo.toml | 2 +- usecase/Cargo.toml | 2 +- 15 files changed, 173 insertions(+), 27 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 14019ac26..7a1858c8f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,149 @@ +# Changelog + +All notable changes to this project will be documented in this file. + +## [0.5.0] - 2025-03-04 +[0.5.0]: https://github.com/metaplex-foundation/aura/compare/v0.1.0...v0.5.0 + +### Bug Fixes + +- Drop core indexing from make file ([ddd1467](https://github.com/metaplex-foundation/aura/commit/ddd146776fcc63b93cf1ec4f0310f93a87653d69)) + +- Add required permissions for GitHub Container Registry (#410) ([d3e4623](https://github.com/metaplex-foundation/aura/commit/d3e4623567b653846c5db818f961c02f88bace37)) + +- [MTG-1301] Fix Docker workflow to use GitHub Container Registry (#412) ([f22ab9f](https://github.com/metaplex-foundation/aura/commit/f22ab9f9c74e93a5f0478c37ae87e14fc5310345)) + +- Fix the build arg in docker (#417) ([faef18d](https://github.com/metaplex-foundation/aura/commit/faef18d5abc07663e5e722265c4a75978a2b07c9)) + +- Encase version info build arg in quotes (#418) ([c9924d9](https://github.com/metaplex-foundation/aura/commit/c9924d938fb77c37d68cf3c399bcf94abf3cc297)) + +- Reorder `cursor` and `options` params in api requests (#411) ([cf54be2](https://github.com/metaplex-foundation/aura/commit/cf54be2f21f80e234e71ce49adc42d69add015d2)) + +- MTG-1225 stop setting asset owner for Fungible tokens in assets_v3 table ([ecbce76](https://github.com/metaplex-foundation/aura/commit/ecbce7622bd4cbe5cf5fb0936513a72b38b62301)) + + +### Build Process & Auxiliary Tools + +- Make build happy ([2a18411](https://github.com/metaplex-foundation/aura/commit/2a184114c53c7610f2a3ac79b801b122d69d6055)) + +- Bump rust toolchain version to 1.84 ([55aa90a](https://github.com/metaplex-foundation/aura/commit/55aa90a6b76e4bfcd7df72499687b18ebc24873e)) + +- Return back old env example because we have example_new ([a96eec8](https://github.com/metaplex-foundation/aura/commit/a96eec8734b5192326299302e1e6da18b9e9039e)) + +- Configure a single sensible rustfmt, format ([9b58797](https://github.com/metaplex-foundation/aura/commit/9b587979e591af9119cca0722cce12d244527215)) + +- Remove unused enum definitions ([a72dfb1](https://github.com/metaplex-foundation/aura/commit/a72dfb16cf42d8e566980218f3beb99ac7f6769c)) + +- Remove `PrintableNft` definition & usages ([4fb61d7](https://github.com/metaplex-foundation/aura/commit/4fb61d7dd6d8a0b737eda3072e1a6c4ea53d94ae)) + +- Fmt ([8868956](https://github.com/metaplex-foundation/aura/commit/886895634919cdbc6257ce018c8dfff03690cbbc)) + +- Regenerate rust files ([59db74e](https://github.com/metaplex-foundation/aura/commit/59db74e8b2156132abdbcc0e329437ac66253d9b)) + +- Add consistency check to ingester.Dockerfile ([70e7466](https://github.com/metaplex-foundation/aura/commit/70e74669b14b30c2e9a05cf4864f7030a7d08f4c)) + +- Remove panic from account processor job start ([841529b](https://github.com/metaplex-foundation/aura/commit/841529b173726035e2d461ea9c3e5f0b2020b160)) + +- Add `start-rocksdb-backup` command to Makefile ([74e4c48](https://github.com/metaplex-foundation/aura/commit/74e4c48a04ec2305ff337fa79c16635e433f10ed)) + +- Mount primary rocksdb path as readonly to backup container ([084b2a9](https://github.com/metaplex-foundation/aura/commit/084b2a9d20110ae4f08c55bbb03582b90f545786)) + +- Optimize retrieval of assets ([f56abf4](https://github.com/metaplex-foundation/aura/commit/f56abf4b6302905d50334c4d17786a4cb7546106)) + +- Move raydium api url to `consts.rs` ([d8beac4](https://github.com/metaplex-foundation/aura/commit/d8beac46473360574d30a3b95367be456707e7b5)) + +- Move out `pprof` as optional, gate behind feature ([75a058a](https://github.com/metaplex-foundation/aura/commit/75a058ab3421db4fc446a425678a0ba5c53b777e)) + +- Fmt rust workflow ([464967c](https://github.com/metaplex-foundation/aura/commit/464967cd687372626392642970da67718d7ae474)) + +- Reorder contributing readme section ([0200e69](https://github.com/metaplex-foundation/aura/commit/0200e694adb2dbd7c2b15694b374266ed5ed2530)) + +- Fix ignored tests (#414) ([c730dac](https://github.com/metaplex-foundation/aura/commit/c730dac99be06417f5ec9bcd09f86563ba879d43)) + +- Added missing cliff config for 0.5 release ([6a67ac1](https://github.com/metaplex-foundation/aura/commit/6a67ac1db830121d2173e95a9dcfa287bb6e071b)) + +- Dropped the version of nft_ingester to the one before 0.5 ([d266f50](https://github.com/metaplex-foundation/aura/commit/d266f50c2aef17175d431f34a1b7cd7895f3e9f6)) + +- Bump rust toolchain to 1.85 (#438) ([9d705e2](https://github.com/metaplex-foundation/aura/commit/9d705e26a9a2edc104d8af4903e8d28819324d1f)) + +- Cherry pick scripts for release ([90c7f51](https://github.com/metaplex-foundation/aura/commit/90c7f51505b922d6c0d3ce7acfda8e3669fe90f2)) + +- Clean cliff.toml ([733aaa7](https://github.com/metaplex-foundation/aura/commit/733aaa7fae8396cced7c2cfeb2c8e00bfa6c5981)) + +- Bump version to 0.5.0 ([09c737f](https://github.com/metaplex-foundation/aura/commit/09c737f15ca1c3cac9434b3c0d70af4e99d227d9)) + + +### CI/CD + +- Enable CI for develop branch PRs (#405) ([85f2336](https://github.com/metaplex-foundation/aura/commit/85f2336974dfc473b9d15908e145942a12591a7e)) + + +### Documentation + +- Add GitFlow contribution guidelines (#402) ([7d044bc](https://github.com/metaplex-foundation/aura/commit/7d044bcc7d0f27f6892ac50c11c8ce696fb6b6e4)) + +- [MTG-1221] Add comprehensive architecture documentation (#420) ([d68d0ac](https://github.com/metaplex-foundation/aura/commit/d68d0acf4bec52b42ad51cb51a2d3c28251797e6)) + + +### Features + +- Update rust versions in dockerfiles & rust-toolchain.toml ([38f2d98](https://github.com/metaplex-foundation/aura/commit/38f2d985a5851ffee835ed7823b19ca13bd0a05f)) + +- Add env feature for clap ([f52806f](https://github.com/metaplex-foundation/aura/commit/f52806f39bf296efdab6ab73ca0d2ab78f2acd33)) + +- Drop profiling for slot persister ([5092470](https://github.com/metaplex-foundation/aura/commit/509247098d8c672f593374cb069ef4e694a46218)) + +- Change the way workers take envs ([0725a9a](https://github.com/metaplex-foundation/aura/commit/0725a9a3da362d41806ada2fb8ec4342723b4b16)) + +- Change env slightly for secondary DB ([73debce](https://github.com/metaplex-foundation/aura/commit/73debce2016c8a8e108c245c19226377657244b9)) + +- Drop profiling from docker compose for slot persister ([728af36](https://github.com/metaplex-foundation/aura/commit/728af3645166199884d34ebecb13938ac55c632b)) + +- Revert changes ([2d8f4d0](https://github.com/metaplex-foundation/aura/commit/2d8f4d077bd948aa6bef8e7e5a97c6179512f163)) + +- Update env.example ([5fb21b8](https://github.com/metaplex-foundation/aura/commit/5fb21b8f822fde53691d3fc51ebd225ad1c2d5c9)) + +- Add a separate secondary rocks backup service ([1fbfc10](https://github.com/metaplex-foundation/aura/commit/1fbfc101c7dc961397b5de73ca4c3ef54e094c22)) + +- Make the backup service a one-time job ([4d01d0c](https://github.com/metaplex-foundation/aura/commit/4d01d0c4d882b44ef136b2671cc930e61a720938)) + +- Use `PathBuf`s in backup-related functions ([221938b](https://github.com/metaplex-foundation/aura/commit/221938ba5dfb0b6f09da4b56ca33b8ec138151d7)) + +- Add metrics for `RedisReceiver` ([72a9ad0](https://github.com/metaplex-foundation/aura/commit/72a9ad0d8d1f41173e10f64e8a3357c923fdd467)) + +- Extend range of interfaces for supply object display (#382) ([f2a45d7](https://github.com/metaplex-foundation/aura/commit/f2a45d799e8b5458153fcd7308d60d965fe93935)) + +- Add Raydium price fetcher cache warmup for symbols ([39c0457](https://github.com/metaplex-foundation/aura/commit/39c0457624de16f13c709de4822ae060d4951400)) + +- Add exclusive backoff for signature fetch ([68f14ba](https://github.com/metaplex-foundation/aura/commit/68f14ba9a6298c3d958de7e0491adc94574b616f)) + +- Cover json downloader with tests ([e91de01](https://github.com/metaplex-foundation/aura/commit/e91de01b0c463ff585b1fd303d6338c2388a549f)) + +- Reverse backfill from the newest slots to the oldest (#407) ([9bda337](https://github.com/metaplex-foundation/aura/commit/9bda337e01ebb44a61bad7dee9df7b4b12b2e2dd)) + +- Rework Dockerfiles and improve build ([2324b6c](https://github.com/metaplex-foundation/aura/commit/2324b6ca06811f79161d2ceb5ea9ca87750689f7)) + +- Add docker build workflow ([4a66616](https://github.com/metaplex-foundation/aura/commit/4a66616ad5e9d5f0ae79e8237d44dda93ea1277b)) + +- Add version info to images ([9fd5ace](https://github.com/metaplex-foundation/aura/commit/9fd5ace276bda32a673f4140d2d8a97feb580533)) + +- Add repository dispatch to docker job ([5353d58](https://github.com/metaplex-foundation/aura/commit/5353d58a3de6f584d1e44d7e74ae08303370232d)) + +- Update blockbuster, bubblegum-batch-sdk to fix acct decode panic (#419) ([b01a695](https://github.com/metaplex-foundation/aura/commit/b01a69548433703e695bdc6ef526800c01227fca)) + +- Rework payload parsing (#423) ([e18e809](https://github.com/metaplex-foundation/aura/commit/e18e809fed02eaf49b35364e67612510938c8b8d)) + +- Improve env variables ([fcfadc3](https://github.com/metaplex-foundation/aura/commit/fcfadc3df60e3efae777d34fe84add45e340967b)) + + +### Testing + +- Add new getAsset test for regular nft collection (#377) ([daac082](https://github.com/metaplex-foundation/aura/commit/daac082cd0b16269baba2442e534cc12a3fdff00)) + +- MTG-1225 test searchAssets by owner with ShowZeroBalance: false ([c69a647](https://github.com/metaplex-foundation/aura/commit/c69a6474af1aa1c696a54bb1ca305cb71ab30659)) + +- MTG-1225 All different search query scenarios for the showZeroBalance option are covered for all token types ([ba768bd](https://github.com/metaplex-foundation/aura/commit/ba768bdb9410e2a01394ce657d82cf4b90a6ae19)) + ## [Unreleased] diff --git a/Cargo.lock b/Cargo.lock index 13ff0005a..be950aa35 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1282,7 +1282,7 @@ dependencies = [ [[package]] name = "backfill-rpc" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "async-trait", "entities", @@ -2082,7 +2082,7 @@ dependencies = [ [[package]] name = "consistency_check" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "bincode", "clap 4.5.26", @@ -2758,7 +2758,7 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "entities" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "base64 0.21.7", "blockbuster", @@ -3311,7 +3311,7 @@ dependencies = [ [[package]] name = "grpc" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "async-trait", "bincode", @@ -3843,7 +3843,7 @@ dependencies = [ [[package]] name = "integration_tests" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "anyhow", "borsh 0.10.3", @@ -3882,7 +3882,7 @@ dependencies = [ [[package]] name = "integrity-verification" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "anchor-lang 0.30.1", "assert-json-diff", @@ -3915,7 +3915,7 @@ dependencies = [ [[package]] name = "interface" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "anchor-lang 0.30.1", "async-trait", @@ -4453,7 +4453,7 @@ dependencies = [ [[package]] name = "metrics-utils" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "chrono", "hyper", @@ -4783,7 +4783,7 @@ dependencies = [ [[package]] name = "nft_ingester" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "anchor-lang 0.30.1", "arweave-rs", @@ -5621,7 +5621,7 @@ dependencies = [ [[package]] name = "postgre-client" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "async-trait", "base64 0.21.7", @@ -6521,7 +6521,7 @@ dependencies = [ [[package]] name = "rocks-db" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "Inflector", "async-channel 2.3.1", @@ -7123,7 +7123,7 @@ dependencies = [ [[package]] name = "setup" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "entities", "flatbuffers 24.3.25", @@ -10371,7 +10371,7 @@ dependencies = [ [[package]] name = "usecase" -version = "0.1.0" +version = "0.5.1-dev" dependencies = [ "anchor-lang 0.30.1", "async-recursion 1.1.1", diff --git a/backfill_rpc/Cargo.toml b/backfill_rpc/Cargo.toml index 225aa998e..8b87cdb46 100644 --- a/backfill_rpc/Cargo.toml +++ b/backfill_rpc/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "backfill-rpc" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/consistency_check/Cargo.toml b/consistency_check/Cargo.toml index e72d26f53..6dbef48db 100644 --- a/consistency_check/Cargo.toml +++ b/consistency_check/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "consistency_check" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/entities/Cargo.toml b/entities/Cargo.toml index 5b33cf387..a9a84fa08 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "entities" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/grpc/Cargo.toml b/grpc/Cargo.toml index 6354b88fb..4bb5a3c30 100644 --- a/grpc/Cargo.toml +++ b/grpc/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "grpc" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index a592fec38..5cb922eb7 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -2,7 +2,7 @@ edition = "2021" name = "integration_tests" publish = false -version = "0.1.0" +version = "0.5.1-dev" [dependencies] anyhow = { workspace = true } diff --git a/integrity_verification/Cargo.toml b/integrity_verification/Cargo.toml index d3da50b96..489f1b5f1 100644 --- a/integrity_verification/Cargo.toml +++ b/integrity_verification/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "integrity-verification" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/interface/Cargo.toml b/interface/Cargo.toml index b8363e805..bf4346a97 100644 --- a/interface/Cargo.toml +++ b/interface/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "interface" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/metrics_utils/Cargo.toml b/metrics_utils/Cargo.toml index dbf7c136c..0d4fd077e 100644 --- a/metrics_utils/Cargo.toml +++ b/metrics_utils/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "metrics-utils" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html @@ -11,4 +11,4 @@ hyper = { workspace = true } chrono = { workspace = true } thiserror = { workspace = true } tracing = { workspace = true } -tokio = { workspace = true } \ No newline at end of file +tokio = { workspace = true } diff --git a/nft_ingester/Cargo.toml b/nft_ingester/Cargo.toml index 58b1858f2..c98df4f4b 100644 --- a/nft_ingester/Cargo.toml +++ b/nft_ingester/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "nft_ingester" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" publish = false diff --git a/postgre-client/Cargo.toml b/postgre-client/Cargo.toml index a7bf91ea6..50c3cd5b5 100644 --- a/postgre-client/Cargo.toml +++ b/postgre-client/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "postgre-client" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/rocks-db/Cargo.toml b/rocks-db/Cargo.toml index d076bb758..29df05465 100644 --- a/rocks-db/Cargo.toml +++ b/rocks-db/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rocks-db" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/tests/setup/Cargo.toml b/tests/setup/Cargo.toml index 8929c3688..bd9756f2f 100644 --- a/tests/setup/Cargo.toml +++ b/tests/setup/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "setup" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/usecase/Cargo.toml b/usecase/Cargo.toml index e5abe9dad..6a969bf0b 100644 --- a/usecase/Cargo.toml +++ b/usecase/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "usecase" -version = "0.1.0" +version = "0.5.1-dev" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html From 314a43737e073d2fc1be228e93b35efce4161b43 Mon Sep 17 00:00:00 2001 From: Olexander Bezsmertnyi Date: Wed, 5 Mar 2025 19:17:08 +0200 Subject: [PATCH 16/21] chore: Add packages cleanup (#443) * [OBEZS] Add cleanup packages; Add DUMP_SYNCHRONIZER_BATCH_SIZE=10000 and SKIP_CHECK_TREE_GAPS=true to .env.example * [OBEZS] Add dry-run cleanup packages and organization * [OBEZS] Add new line character at the end of file * [OBEZS] Set stop_grace_period to 2m for ingester * chore: Update default value of the dump_synchronizer_batch_size --------- Co-authored-by: obezsmertnyi Co-authored-by: Stanislav Cherviakov --- .env.example | 28 ++++---- .github/workflows/cleanup.yml | 31 +++++++++ .../workflows/delete-container-versions.yml | 66 +++++++++++++++++++ docker-compose.yaml | 2 +- nft_ingester/src/config.rs | 4 +- 5 files changed, 116 insertions(+), 15 deletions(-) create mode 100644 .github/workflows/cleanup.yml create mode 100644 .github/workflows/delete-container-versions.yml diff --git a/.env.example b/.env.example index 137e2e737..b6d6da544 100644 --- a/.env.example +++ b/.env.example @@ -3,16 +3,16 @@ DOCKER_IMAGE_TAG=develop-latest RPC_HOST='https://mainnet-aura.metaplex.com/{personal_rpc_key}' #Configurable app parts that could be enabled or disabled. (values in the default positions) +CHECK_PROOFS=false +ENABLE_ROCKS_MIGRATION=true +RESTORE_ROCKS_DB=false RUN_API=true RUN_BACKFILLER=true RUN_BUBBLEGUM_BACKFILLER=true RUN_GAPFILLER=false -SHOULD_REINGEST=false RUN_PROFILING=false -RESTORE_ROCKS_DB=false -ENABLE_ROCKS_MIGRATION=true -CHECK_PROOFS=false -SKIP_CHECK_TREE_GAPS=false +SHOULD_REINGEST=false +SKIP_CHECK_TREE_GAPS=true #Postgres PG_DATABASE_URL='postgres://solana:solana@localhost:5432/aura_db' @@ -26,28 +26,32 @@ ROCKS_DB_PATH="/usr/src/rocksdb-data" ROCKS_DB_SECONDARY_PATH="/path/to/rocks/secondary/db" # path to the slots data, required for the backfiller to work ROCKS_SLOTS_DB_PATH=/path/to/slots-data -ROCKS_SECONDARY_SLOTS_DB_PATH=/path/to/secondary/ingester-slots + ROCKS_ARCHIVES_DIR="/path/to/rocks/backup/archives" ROCKS_BACKUP_ARCHIVES_DIR="/path/to/rocks/backup/archives" -ROCKS_BACKUP_URL="127.0.0.1:3051/snapshot" -ROCKS_MIGRATION_STORAGE_PATH=/path/to/migration_storage ROCKS_BACKUP_DIR="/path/to/rocks/backup/" +ROCKS_BACKUP_URL="127.0.0.1:3051/snapshot" ROCKS_DUMP_PATH="/path/to/dump" +ROCKS_MIGRATION_STORAGE_PATH=/path/to/migration_storage +ROCKS_SECONDARY_SLOTS_DB_PATH=/path/to/secondary/ingester-slots #Backfiller BACKFILLER_SOURCE_MODE=bigtable BIG_TABLE_CONFIG='{"creds": "/usr/src/app/creds.json", "timeout": 1000}' +# Synchronizer +DUMP_SYNCHRONIZER_BATCH_SIZE=10000 + +# API server port (if API is enabled) +API_SERVER_PORT=8990 +INGESTER_SERVER_PORT=9092 + # Metrics port. Start HTTP server to report metrics if port exist. API_METRICS_PORT=8985 INGESTER_METRICS_PORT=9091 MIGRATOR_METRICS_PORT=5091 SYNCHRONIZER_METRICS_PORT=6091 -# API server port (if API is enabled) -INGESTER_SERVER_PORT=9092 -API_SERVER_PORT=8990 - # GRPC Server port PEER_GRPC_PORT=9099 GAPFILLER_PEER_ADDR="0.0.0.0" diff --git a/.github/workflows/cleanup.yml b/.github/workflows/cleanup.yml new file mode 100644 index 000000000..bcd4fefe2 --- /dev/null +++ b/.github/workflows/cleanup.yml @@ -0,0 +1,31 @@ +name: Cleanup Container Images + +on: + workflow_dispatch: + schedule: + - cron: '0 3 * * 0' + +jobs: + cleanup: + runs-on: ubuntu-latest + env: + REPO_NAME: ${GITHUB_REPOSITORY#*/} + strategy: + matrix: + binary: [ingester, das-api, slot_persister, backfill, synchronizer, rocksdb_backup] + steps: + - name: Set full package name + id: set_pkg + run: | + FULL_NAME="${REPO_NAME}-${{ matrix.binary }}" + echo "package_name=${FULL_NAME}" >> $GITHUB_OUTPUT + echo "Cleaning up package: ${FULL_NAME}" + + - name: Cleanup package images + uses: ./.github/workflows/delete-container-versions.yml + with: + package_name: ${{ steps.set_pkg.outputs.package_name }} + keep_release: 30 + keep_develop: 20 + keep_other: 20 + keep_untagged: 0 diff --git a/.github/workflows/delete-container-versions.yml b/.github/workflows/delete-container-versions.yml new file mode 100644 index 000000000..23d7542b8 --- /dev/null +++ b/.github/workflows/delete-container-versions.yml @@ -0,0 +1,66 @@ +name: Delete Container Versions + +on: + workflow_call: + inputs: + package_name: + required: true + type: string + keep_release: + required: false + type: number + default: 30 + keep_develop: + required: false + type: number + default: 20 + keep_other: + required: false + type: number + default: 20 + keep_untagged: + required: false + type: number + default: 0 + +jobs: + delete_old_packages: + runs-on: ubuntu-latest + steps: + - name: Keep last release versions + uses: smartsquaregmbh/delete-old-packages@v0.8.1 + with: + organization: ${{ github.repository_owner }} + type: container + names: ${{ inputs.package_name }} + version-pattern: '^v\d+\.\d+\.\d+$' + keep: ${{ inputs.keep_release }} + dry-run: true + + - name: Keep last develop versions + uses: smartsquaregmbh/delete-old-packages@v0.8.1 + with: + organization: ${{ github.repository_owner }} + type: container + names: ${{ inputs.package_name }} + version-pattern: '^develop-.*' + keep: ${{ inputs.keep_develop }} + dry-run: true + + - name: Keep last other tagged versions + uses: smartsquaregmbh/delete-old-packages@v0.8.1 + with: + organization: ${{ github.repository_owner }} + type: container + names: ${{ inputs.package_name }} + version-pattern: '^(?!v\d+\.\d+\.\d+$)(?!latest)(?!develop-).*' + keep: ${{ inputs.keep_other }} + dry-run: true + + - name: Keep last untagged versions + uses: actions/delete-package-versions@v5 + with: + package-name: ${{ inputs.package_name }} + package-type: container + delete-only-untagged-versions: 'true' + min-versions-to-keep: ${{ inputs.keep_untagged }} diff --git a/docker-compose.yaml b/docker-compose.yaml index 2aa4c6ff8..b63c7591a 100644 --- a/docker-compose.yaml +++ b/docker-compose.yaml @@ -23,7 +23,7 @@ services: # - ${PROFILING_FILE_PATH}:${PROFILING_FILE_PATH_CONTAINER}:rw depends_on: - db - stop_grace_period: 10m + stop_grace_period: 2m logging: options: max-size: "2048m" diff --git a/nft_ingester/src/config.rs b/nft_ingester/src/config.rs index 231ca63e4..1f28919b1 100644 --- a/nft_ingester/src/config.rs +++ b/nft_ingester/src/config.rs @@ -313,7 +313,7 @@ pub struct SynchronizerClapArgs { pub metrics_port: Option, pub profiling_file_path_container: Option, - #[clap(long, env, default_value = "200000")] + #[clap(long, env, default_value = "10000")] pub dump_synchronizer_batch_size: usize, #[clap( long, @@ -639,7 +639,7 @@ mod tests { assert_eq!(args.rocks_db_secondary_path, "./my_rocksdb_secondary"); assert_eq!(args.run_profiling, false); assert_eq!(args.heap_path, "/usr/src/app/heaps"); - assert_eq!(args.dump_synchronizer_batch_size, 200000); + assert_eq!(args.dump_synchronizer_batch_size, 10000); assert_eq!(args.dump_sync_threshold, 150000000); assert_eq!(args.synchronizer_parallel_tasks, 30); assert_eq!(args.timeout_between_syncs_sec, 0); From 333dbcd4092076695951820a3a371cf1eb550290 Mon Sep 17 00:00:00 2001 From: Andrii Date: Thu, 6 Mar 2025 13:11:03 +0100 Subject: [PATCH 17/21] [mtg-1380] fix(api): getAssetsByCreator doesn't return empty fields (#444) * MTG-1380 getAssetsByCreator doesn't return empty fields. - Return empty string for all collection_metadata fields if we have no values --- Cargo.lock | 2 +- nft_ingester/Cargo.toml | 2 +- nft_ingester/docker-compose_db.yml | 4 +- .../src/api/dapi/rpc_asset_convertors.rs | 82 +++++++++++++++++-- nft_ingester/tests/api_tests.rs | 17 ++++ 5 files changed, 96 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index be950aa35..4d22b7661 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4787,6 +4787,7 @@ version = "0.5.1-dev" dependencies = [ "anchor-lang 0.30.1", "arweave-rs", + "assert-json-diff", "assertables", "async-channel 2.3.1", "async-trait", @@ -4838,7 +4839,6 @@ dependencies = [ "rocksdb", "schemars", "serde", - "serde_cbor", "serde_json", "setup", "solana-client", diff --git a/nft_ingester/Cargo.toml b/nft_ingester/Cargo.toml index c98df4f4b..1d172b6a3 100644 --- a/nft_ingester/Cargo.toml +++ b/nft_ingester/Cargo.toml @@ -66,7 +66,6 @@ xxhash-rust = { workspace = true } hex = { workspace = true } jemallocator = { workspace = true } arweave-rs = { workspace = true } -serde_cbor = { workspace = true } bubblegum-batch-sdk = { workspace = true } jsonpath_lib = { workspace = true } mime_guess = { workspace = true } @@ -81,6 +80,7 @@ axum = { workspace = true } rocksdb = { workspace = true } num-bigint = "0.4" tracing-test = { workspace = true } +assert-json-diff = "2.0" [dev-dependencies] setup = { path = "../tests/setup" } diff --git a/nft_ingester/docker-compose_db.yml b/nft_ingester/docker-compose_db.yml index 1a4cdf11f..1f94d1ea6 100644 --- a/nft_ingester/docker-compose_db.yml +++ b/nft_ingester/docker-compose_db.yml @@ -2,11 +2,11 @@ version: "3.7" services: pgdb: image: "postgres:14.15" - container_name: pgdb + container_name: pg_db environment: - POSTGRES_USER=solana - POSTGRES_PASSWORD=solana - - POSTGRES_DB=aura_db + - POSTGRES_DB=solana ports: - "5432:5432" diff --git a/nft_ingester/src/api/dapi/rpc_asset_convertors.rs b/nft_ingester/src/api/dapi/rpc_asset_convertors.rs index ba75a46c9..3917c3a22 100644 --- a/nft_ingester/src/api/dapi/rpc_asset_convertors.rs +++ b/nft_ingester/src/api/dapi/rpc_asset_convertors.rs @@ -234,16 +234,23 @@ fn extract_collection_metadata( meta.set_item("name", name.clone()); } - let symbol = safe_select(chain_data_selector, "$.symbol"); - if let Some(symbol) = symbol { - meta.set_item("symbol", symbol.clone()); + let link_fields = vec!["name", "symbol"]; + for name in link_fields { + let value = safe_select(chain_data_selector, format!("$.{}", name).as_str()); + if let Some(symbol) = value { + meta.set_item(name, symbol.clone()); + } else { + meta.set_item(name, "".into()); + } } let link_fields = vec!["image", "external_url", "description"]; - for f in link_fields { - let l = safe_select(selector, format!("$.{}", f).as_str()); - if let Some(l) = l { - meta.set_item(f, l.to_owned()); + for name in link_fields { + let value = safe_select(selector, format!("$.{}", name).as_str()); + if let Some(symbol) = value { + meta.set_item(name, symbol.clone()); + } else { + meta.set_item(name, "".into()); } } @@ -629,3 +636,64 @@ fn get_pagination_values( Ok(Pagination { after: last_row, before: first_row, ..Default::default() }) } } + +#[cfg(test)] +mod tests { + use assert_json_diff::assert_json_eq; + use entities::models::Updated; + use serde_json::json; + + use super::*; + + #[test] + fn test_extract_collection_metadata() { + let asset_dynamic = AssetDynamicDetails { + url: Updated::new(12, None, "https://example.com".to_string()), + onchain_data: Some(Updated::new( + 12, + None, + r#"{"name": "Test Asset", "symbol": "TST"}"#.to_string(), + )), + ..Default::default() + }; + let offchain_data = OffChainData { + metadata: Some(r#"{"image": "https://example.com/image.png", "external_url": "https://example.com", "description": "Test Description"}"#.to_string()), + ..Default::default() + }; + + let expected_metadata = json!({ + "name": "Test Asset", + "symbol": "TST", + "image": "https://example.com/image.png", + "external_url": "https://example.com", + "description": "Test Description" + }); + + let result = extract_collection_metadata(&asset_dynamic, &offchain_data); + + assert_json_eq!(result, expected_metadata); + } + + #[test] + fn test_extract_collection_metadata_empty_fields() { + let asset_dynamic = AssetDynamicDetails { + url: Updated::new(12, None, "https://example.com".to_string()), + onchain_data: Some(Updated::new(12, None, r#"{}"#.to_string())), + ..Default::default() + }; + let offchain_data = + OffChainData { metadata: Some(r#"{}"#.to_string()), ..Default::default() }; + + let expected_metadata = json!({ + "name": "", + "symbol": "", + "image": "", + "external_url": "", + "description": "" + }); + + let result = extract_collection_metadata(&asset_dynamic, &offchain_data); + + assert_json_eq!(result, expected_metadata); + } +} diff --git a/nft_ingester/tests/api_tests.rs b/nft_ingester/tests/api_tests.rs index cf46cdb0d..719c7cfc5 100644 --- a/nft_ingester/tests/api_tests.rs +++ b/nft_ingester/tests/api_tests.rs @@ -4170,6 +4170,23 @@ mod tests { cursor: None } ); + + // getAssetsByCreator V1 + let request_params = r#"["DASPQfEAVcHp55eFmfstRduMT3dSfoTirFFsMHwUaWaz",true,null,null,1,null,null,{"showCollectionMetadata":true},null]"#; + let rpc_params: jsonrpc_core::Params = + serde_json::from_str(request_params).expect("params are valid json"); + let params_deserialized: GetAssetsByCreator = rpc_params + .parse() + .expect("params provided deserialize correctly into GetAssetsByOwner"); + + assert_eq!( + params_deserialized.creator_address, + "DASPQfEAVcHp55eFmfstRduMT3dSfoTirFFsMHwUaWaz".to_owned() + ); + assert_eq!(params_deserialized.only_verified, Some(true)); + assert_eq!(params_deserialized.options.show_collection_metadata, true); + assert_eq!(params_deserialized.page, Some(1)); + // getAssets let request_params = r#"[["asset1","asset2"]]"#; let rpc_params: jsonrpc_core::Params = From 370d3d32297443b9f1d414f21f2b6dfea031307a Mon Sep 17 00:00:00 2001 From: Oleksandr Mykhailenko <58030797+armyhaylenko@users.noreply.github.com> Date: Thu, 6 Mar 2025 17:17:16 +0200 Subject: [PATCH 18/21] fix(nft_ingester): make backfill not wait for CTRL-C (#445) --- nft_ingester/src/bin/backfill/main.rs | 41 +++++++-------------------- 1 file changed, 11 insertions(+), 30 deletions(-) diff --git a/nft_ingester/src/bin/backfill/main.rs b/nft_ingester/src/bin/backfill/main.rs index 0c1741ca1..b1816abf1 100644 --- a/nft_ingester/src/bin/backfill/main.rs +++ b/nft_ingester/src/bin/backfill/main.rs @@ -63,10 +63,19 @@ async fn main() { .expect("Failed to open source RocksDB"); let red_metrics = Arc::new(RequestErrorDurationMetrics::new()); let cancellation_token = CancellationToken::new(); - let stop_handle = tokio::task::spawn({ + + // Concurrency setup + let num_workers = args.workers; + let (slot_sender, slot_receiver) = async_channel::bounded::<(u64, Vec)>(num_workers * 2); + let slots_processed = Arc::new(AtomicU64::new(0)); + let rate = Arc::new(Mutex::new(0.0)); + + tokio::task::spawn({ let cancellation_token = cancellation_token.clone(); + let slot_sender = slot_sender.clone(); async move { usecase::graceful_stop::graceful_shutdown(cancellation_token).await; + slot_sender.close(); } }); @@ -89,31 +98,6 @@ async fn main() { let consumer = Arc::new(DirectBlockParser::new(tx_ingester.clone(), target_db.clone(), metrics.clone())); - // Concurrency setup - let num_workers = args.workers; - let (slot_sender, slot_receiver) = async_channel::bounded::<(u64, Vec)>(num_workers * 2); - let slots_processed = Arc::new(AtomicU64::new(0)); - let rate = Arc::new(Mutex::new(0.0)); - - usecase::executor::spawn({ - let cancellation_token = cancellation_token.clone(); - let slot_sender = slot_sender.clone(); - async move { - // Wait for Ctrl+C signal - match tokio::signal::ctrl_c().await { - Ok(()) => { - info!("Received Ctrl+C, shutting down gracefully..."); - cancellation_token.cancel(); - // Close the channel to signal workers to stop - slot_sender.close(); - }, - Err(err) => { - error!("Unable to listen for shutdown signal: {}", err); - }, - } - } - }); - // Parse slots if provided let mut slots_to_process = Vec::new(); if let Some(slots_str) = args.slots { @@ -244,7 +228,7 @@ async fn main() { worker_handles.push(handle); } - usecase::executor::spawn({ + tokio::task::spawn({ let cancellation_token = cancellation_token.clone(); let slots_processed = slots_processed.clone(); let rate = rate.clone(); @@ -310,9 +294,6 @@ async fn main() { let _ = handle.await; } - if stop_handle.await.is_err() { - error!("Error joining graceful shutdown!"); - } progress_bar.finish_with_message("Processing complete"); } From 98b517fe55c1ac67c1f795c6dd1c9bfc5721b716 Mon Sep 17 00:00:00 2001 From: Stanislav Cherviakov Date: Mon, 10 Mar 2025 10:14:08 +0000 Subject: [PATCH 19/21] fix(rocksdb): prioritize slot ordering over version in comparison logic [MTG-1415] (#446) * fix(rocksdb): prioritize slot ordering over version in comparison logic Change the comparison order in macro to check slot_updated values first before checking update_version. This ensures account-based assets are properly indexed by slot-first ordering, which better represents the correct temporal sequence of asset updates. Closes MTG-1415 * chore: fmt * test: Fix tests after reprioritising the ordering --- rocks-db/src/columns/asset.rs | 17 +++++++++-------- rocks-db/src/mappers.rs | 20 +++++++++++++------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/rocks-db/src/columns/asset.rs b/rocks-db/src/columns/asset.rs index 5db37bd4c..d3c54a770 100644 --- a/rocks-db/src/columns/asset.rs +++ b/rocks-db/src/columns/asset.rs @@ -3607,7 +3607,12 @@ mod tests { } assert!(asset.other_known_owners().is_none()); let asset_mapped = AssetCompleteDetails::from(asset); - assert_eq!(asset_mapped.owner.unwrap().owner.value.unwrap(), new_owner); + // Now that slot is prioritized over version, we expect the existing owner to be kept + // because the new owner has a smaller slot (1) despite higher write version + assert_eq!( + asset_mapped.owner.as_ref().unwrap().owner.value.unwrap(), + Pubkey::from_str(EXISTING_OWNER).unwrap() + ); } #[test] @@ -3650,13 +3655,9 @@ mod tests { } assert!(asset.other_known_owners().is_none()); let asset_mapped = AssetCompleteDetails::from(asset); - assert_eq!( - asset_mapped.owner.as_ref().unwrap().owner.value.unwrap(), - Pubkey::from_str(EXISTING_OWNER).unwrap() - ); - // This is ther case, when the is current owner was not ever set and now it's updated by some old update. - // This update shouldn't have happened as usually the vesioning of owner, delegate and is_current_owner is done together. - // For the case of empty data after the migration this is acceptable imho. + // With slot prioritized over version, the new owner should be used because it has a higher slot + // even though it has a smaller write version + assert_eq!(asset_mapped.owner.as_ref().unwrap().owner.value.unwrap(), new_owner); assert!(asset_mapped.owner.unwrap().is_current_owner.value); } diff --git a/rocks-db/src/mappers.rs b/rocks-db/src/mappers.rs index e8e2177d3..6fbd2ec7c 100644 --- a/rocks-db/src/mappers.rs +++ b/rocks-db/src/mappers.rs @@ -228,13 +228,19 @@ macro_rules! impl_partial_ord_for_updated { ($name:ident) => { impl PartialOrd for fb::$name<'_> { fn partial_cmp(&self, other: &Self) -> Option { - Some(match self.update_version().partial_cmp(&other.update_version()) { - Some(std::cmp::Ordering::Equal) => { - self.slot_updated().cmp(&other.slot_updated()) - }, - Some(ord) => ord, - None => self.slot_updated().cmp(&other.slot_updated()), - }) + // Compare slot_updated first + let slot_order = self.slot_updated().cmp(&other.slot_updated()); + + // If slots are equal, then check update_version + if slot_order == Ordering::Equal { + match self.update_version().partial_cmp(&other.update_version()) { + Some(ord) => Some(ord), + None => Some(Ordering::Equal), // If versions can't be compared, consider them equal since slots are equal + } + } else { + // If slots are different, use that ordering + Some(slot_order) + } } } }; From 47d4aab1818f6fb34fc3196ff111427bed4aaeca Mon Sep 17 00:00:00 2001 From: GitHub Actions Date: Mon, 10 Mar 2025 12:09:26 +0000 Subject: [PATCH 20/21] chore: bump version to 0.6.0 --- Cargo.lock | 26 +++++++++++++------------- backfill_rpc/Cargo.toml | 2 +- consistency_check/Cargo.toml | 2 +- entities/Cargo.toml | 2 +- grpc/Cargo.toml | 2 +- integration_tests/Cargo.toml | 2 +- integrity_verification/Cargo.toml | 2 +- interface/Cargo.toml | 2 +- metrics_utils/Cargo.toml | 2 +- nft_ingester/Cargo.toml | 2 +- postgre-client/Cargo.toml | 2 +- rocks-db/Cargo.toml | 2 +- tests/setup/Cargo.toml | 2 +- usecase/Cargo.toml | 2 +- 14 files changed, 26 insertions(+), 26 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4d22b7661..4063e08c0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1282,7 +1282,7 @@ dependencies = [ [[package]] name = "backfill-rpc" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "async-trait", "entities", @@ -2082,7 +2082,7 @@ dependencies = [ [[package]] name = "consistency_check" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "bincode", "clap 4.5.26", @@ -2758,7 +2758,7 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "entities" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "base64 0.21.7", "blockbuster", @@ -3311,7 +3311,7 @@ dependencies = [ [[package]] name = "grpc" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "async-trait", "bincode", @@ -3843,7 +3843,7 @@ dependencies = [ [[package]] name = "integration_tests" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "anyhow", "borsh 0.10.3", @@ -3882,7 +3882,7 @@ dependencies = [ [[package]] name = "integrity-verification" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "anchor-lang 0.30.1", "assert-json-diff", @@ -3915,7 +3915,7 @@ dependencies = [ [[package]] name = "interface" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "anchor-lang 0.30.1", "async-trait", @@ -4453,7 +4453,7 @@ dependencies = [ [[package]] name = "metrics-utils" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "chrono", "hyper", @@ -4783,7 +4783,7 @@ dependencies = [ [[package]] name = "nft_ingester" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "anchor-lang 0.30.1", "arweave-rs", @@ -5621,7 +5621,7 @@ dependencies = [ [[package]] name = "postgre-client" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "async-trait", "base64 0.21.7", @@ -6521,7 +6521,7 @@ dependencies = [ [[package]] name = "rocks-db" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "Inflector", "async-channel 2.3.1", @@ -7123,7 +7123,7 @@ dependencies = [ [[package]] name = "setup" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "entities", "flatbuffers 24.3.25", @@ -10371,7 +10371,7 @@ dependencies = [ [[package]] name = "usecase" -version = "0.5.1-dev" +version = "0.6.0" dependencies = [ "anchor-lang 0.30.1", "async-recursion 1.1.1", diff --git a/backfill_rpc/Cargo.toml b/backfill_rpc/Cargo.toml index 8b87cdb46..ad94aa9ad 100644 --- a/backfill_rpc/Cargo.toml +++ b/backfill_rpc/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "backfill-rpc" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/consistency_check/Cargo.toml b/consistency_check/Cargo.toml index 6dbef48db..af090a9d2 100644 --- a/consistency_check/Cargo.toml +++ b/consistency_check/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "consistency_check" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/entities/Cargo.toml b/entities/Cargo.toml index a9a84fa08..db22105a9 100644 --- a/entities/Cargo.toml +++ b/entities/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "entities" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/grpc/Cargo.toml b/grpc/Cargo.toml index 4bb5a3c30..57ce5eead 100644 --- a/grpc/Cargo.toml +++ b/grpc/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "grpc" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/integration_tests/Cargo.toml b/integration_tests/Cargo.toml index 5cb922eb7..3dab33951 100644 --- a/integration_tests/Cargo.toml +++ b/integration_tests/Cargo.toml @@ -2,7 +2,7 @@ edition = "2021" name = "integration_tests" publish = false -version = "0.5.1-dev" +version = "0.6.0" [dependencies] anyhow = { workspace = true } diff --git a/integrity_verification/Cargo.toml b/integrity_verification/Cargo.toml index 489f1b5f1..3ff2367b0 100644 --- a/integrity_verification/Cargo.toml +++ b/integrity_verification/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "integrity-verification" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/interface/Cargo.toml b/interface/Cargo.toml index bf4346a97..1b568516c 100644 --- a/interface/Cargo.toml +++ b/interface/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "interface" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/metrics_utils/Cargo.toml b/metrics_utils/Cargo.toml index 0d4fd077e..4cde476fb 100644 --- a/metrics_utils/Cargo.toml +++ b/metrics_utils/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "metrics-utils" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/nft_ingester/Cargo.toml b/nft_ingester/Cargo.toml index 1d172b6a3..5da01fdd8 100644 --- a/nft_ingester/Cargo.toml +++ b/nft_ingester/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "nft_ingester" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" publish = false diff --git a/postgre-client/Cargo.toml b/postgre-client/Cargo.toml index 50c3cd5b5..ece9e5429 100644 --- a/postgre-client/Cargo.toml +++ b/postgre-client/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "postgre-client" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/rocks-db/Cargo.toml b/rocks-db/Cargo.toml index 29df05465..47d205fe7 100644 --- a/rocks-db/Cargo.toml +++ b/rocks-db/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "rocks-db" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/tests/setup/Cargo.toml b/tests/setup/Cargo.toml index bd9756f2f..b914895a4 100644 --- a/tests/setup/Cargo.toml +++ b/tests/setup/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "setup" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html diff --git a/usecase/Cargo.toml b/usecase/Cargo.toml index 6a969bf0b..cf68be4db 100644 --- a/usecase/Cargo.toml +++ b/usecase/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "usecase" -version = "0.5.1-dev" +version = "0.6.0" edition = "2021" # See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html From 9658bc4daebb2e8ba3ec62e21db6d7b714541185 Mon Sep 17 00:00:00 2001 From: GitHub Actions Date: Mon, 10 Mar 2025 12:09:26 +0000 Subject: [PATCH 21/21] docs: add changelog for v0.6.0 --- CHANGELOG.md | 164 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 164 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7a1858c8f..2bde77d28 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -2,6 +2,170 @@ All notable changes to this project will be documented in this file. +## [0.6.0] - 2025-03-10 +[0.6.0]: https://github.com/metaplex-foundation/aura/compare/v0.5.0...v0.6.0 + +### Bug Fixes + +- Drop core indexing from make file ([ddd1467](https://github.com/metaplex-foundation/aura/commit/ddd146776fcc63b93cf1ec4f0310f93a87653d69)) + +- Add required permissions for GitHub Container Registry (#410) ([d3e4623](https://github.com/metaplex-foundation/aura/commit/d3e4623567b653846c5db818f961c02f88bace37)) + +- [MTG-1301] Fix Docker workflow to use GitHub Container Registry (#412) ([f22ab9f](https://github.com/metaplex-foundation/aura/commit/f22ab9f9c74e93a5f0478c37ae87e14fc5310345)) + +- Fix the build arg in docker (#417) ([faef18d](https://github.com/metaplex-foundation/aura/commit/faef18d5abc07663e5e722265c4a75978a2b07c9)) + +- Encase version info build arg in quotes (#418) ([c9924d9](https://github.com/metaplex-foundation/aura/commit/c9924d938fb77c37d68cf3c399bcf94abf3cc297)) + +- Reorder `cursor` and `options` params in api requests (#411) ([cf54be2](https://github.com/metaplex-foundation/aura/commit/cf54be2f21f80e234e71ce49adc42d69add015d2)) + +- MTG-1225 stop setting asset owner for Fungible tokens in assets_v3 table ([ecbce76](https://github.com/metaplex-foundation/aura/commit/ecbce7622bd4cbe5cf5fb0936513a72b38b62301)) + +- Parse big table config into the correct type (#430) ([946d4e4](https://github.com/metaplex-foundation/aura/commit/946d4e4bb295b65c9c6d2fe46b086c604b041fc6)) + +- Resolve issues with release workflows (#432) ([f49ee30](https://github.com/metaplex-foundation/aura/commit/f49ee305c77f7df3a9e25fe317f44505040bda06)) + +- Change `parse_json` expected type in config (#435) ([411ea91](https://github.com/metaplex-foundation/aura/commit/411ea91fb639c575b3d3180e751ba46c46584f42)) + +- Fix github PR creation in release prepare [MTG-1360] (#439) ([1cbe9f7](https://github.com/metaplex-foundation/aura/commit/1cbe9f797643b36ff44d5c7714abc8d008fcb398)) + +- Make backfill not wait for CTRL-C (#445) ([370d3d3](https://github.com/metaplex-foundation/aura/commit/370d3d32297443b9f1d414f21f2b6dfea031307a)) + +- Prioritize slot ordering over version in comparison logic [MTG-1415] (#446) ([98b517f](https://github.com/metaplex-foundation/aura/commit/98b517fe55c1ac67c1f795c6dd1c9bfc5721b716)) + + +### Build Process & Auxiliary Tools + +- Make build happy ([2a18411](https://github.com/metaplex-foundation/aura/commit/2a184114c53c7610f2a3ac79b801b122d69d6055)) + +- Bump rust toolchain version to 1.84 ([55aa90a](https://github.com/metaplex-foundation/aura/commit/55aa90a6b76e4bfcd7df72499687b18ebc24873e)) + +- Return back old env example because we have example_new ([a96eec8](https://github.com/metaplex-foundation/aura/commit/a96eec8734b5192326299302e1e6da18b9e9039e)) + +- Configure a single sensible rustfmt, format ([9b58797](https://github.com/metaplex-foundation/aura/commit/9b587979e591af9119cca0722cce12d244527215)) + +- Remove unused enum definitions ([a72dfb1](https://github.com/metaplex-foundation/aura/commit/a72dfb16cf42d8e566980218f3beb99ac7f6769c)) + +- Remove `PrintableNft` definition & usages ([4fb61d7](https://github.com/metaplex-foundation/aura/commit/4fb61d7dd6d8a0b737eda3072e1a6c4ea53d94ae)) + +- Fmt ([8868956](https://github.com/metaplex-foundation/aura/commit/886895634919cdbc6257ce018c8dfff03690cbbc)) + +- Regenerate rust files ([59db74e](https://github.com/metaplex-foundation/aura/commit/59db74e8b2156132abdbcc0e329437ac66253d9b)) + +- Add consistency check to ingester.Dockerfile ([70e7466](https://github.com/metaplex-foundation/aura/commit/70e74669b14b30c2e9a05cf4864f7030a7d08f4c)) + +- Remove panic from account processor job start ([841529b](https://github.com/metaplex-foundation/aura/commit/841529b173726035e2d461ea9c3e5f0b2020b160)) + +- Add `start-rocksdb-backup` command to Makefile ([74e4c48](https://github.com/metaplex-foundation/aura/commit/74e4c48a04ec2305ff337fa79c16635e433f10ed)) + +- Mount primary rocksdb path as readonly to backup container ([084b2a9](https://github.com/metaplex-foundation/aura/commit/084b2a9d20110ae4f08c55bbb03582b90f545786)) + +- Optimize retrieval of assets ([f56abf4](https://github.com/metaplex-foundation/aura/commit/f56abf4b6302905d50334c4d17786a4cb7546106)) + +- Move raydium api url to `consts.rs` ([d8beac4](https://github.com/metaplex-foundation/aura/commit/d8beac46473360574d30a3b95367be456707e7b5)) + +- Move out `pprof` as optional, gate behind feature ([75a058a](https://github.com/metaplex-foundation/aura/commit/75a058ab3421db4fc446a425678a0ba5c53b777e)) + +- Fmt rust workflow ([464967c](https://github.com/metaplex-foundation/aura/commit/464967cd687372626392642970da67718d7ae474)) + +- Reorder contributing readme section ([0200e69](https://github.com/metaplex-foundation/aura/commit/0200e694adb2dbd7c2b15694b374266ed5ed2530)) + +- Fix ignored tests (#414) ([c730dac](https://github.com/metaplex-foundation/aura/commit/c730dac99be06417f5ec9bcd09f86563ba879d43)) + +- Clean up env some more & downsize compose (#425) ([a2472a9](https://github.com/metaplex-foundation/aura/commit/a2472a9df80ccf5be97c9fa82afdf110cd093379)) + +- [MTG-1360] release process (#431) ([b0cbe40](https://github.com/metaplex-foundation/aura/commit/b0cbe4005bfcc56dee4b3d02c4f22bf307da0523)) + +- Modularize release preparation process [MTG-1360] (#436) ([6b1cb34](https://github.com/metaplex-foundation/aura/commit/6b1cb344319db9d77b4682fb568ad71020dbce36)) + +- Bump rust toolchain to 1.85 (#438) ([bf1b557](https://github.com/metaplex-foundation/aura/commit/bf1b55755d7c1a0fd67748eac5b214fb3f9f9a85)) + +- Merge release v0.5 (#442) ([4ae77f2](https://github.com/metaplex-foundation/aura/commit/4ae77f2b7a2fa0aef32396b112e9a1be45da14e6)) + +- Bump version to 0.6.0 ([47d4aab](https://github.com/metaplex-foundation/aura/commit/47d4aab1818f6fb34fc3196ff111427bed4aaeca)) + + +### CI/CD + +- Enable CI for develop branch PRs (#405) ([85f2336](https://github.com/metaplex-foundation/aura/commit/85f2336974dfc473b9d15908e145942a12591a7e)) + + +### Documentation + +- Add GitFlow contribution guidelines (#402) ([7d044bc](https://github.com/metaplex-foundation/aura/commit/7d044bcc7d0f27f6892ac50c11c8ce696fb6b6e4)) + +- [MTG-1221] Add comprehensive architecture documentation (#420) ([d68d0ac](https://github.com/metaplex-foundation/aura/commit/d68d0acf4bec52b42ad51cb51a2d3c28251797e6)) + +- Fixed broken link and grammatical errors ([ea24003](https://github.com/metaplex-foundation/aura/commit/ea24003f3f719ec5c5029f8f1dcbf9ccd10d5bfb)) + + +### Features + +- Update rust versions in dockerfiles & rust-toolchain.toml ([38f2d98](https://github.com/metaplex-foundation/aura/commit/38f2d985a5851ffee835ed7823b19ca13bd0a05f)) + +- Add env feature for clap ([f52806f](https://github.com/metaplex-foundation/aura/commit/f52806f39bf296efdab6ab73ca0d2ab78f2acd33)) + +- Drop profiling for slot persister ([5092470](https://github.com/metaplex-foundation/aura/commit/509247098d8c672f593374cb069ef4e694a46218)) + +- Change the way workers take envs ([0725a9a](https://github.com/metaplex-foundation/aura/commit/0725a9a3da362d41806ada2fb8ec4342723b4b16)) + +- Change env slightly for secondary DB ([73debce](https://github.com/metaplex-foundation/aura/commit/73debce2016c8a8e108c245c19226377657244b9)) + +- Drop profiling from docker compose for slot persister ([728af36](https://github.com/metaplex-foundation/aura/commit/728af3645166199884d34ebecb13938ac55c632b)) + +- Revert changes ([2d8f4d0](https://github.com/metaplex-foundation/aura/commit/2d8f4d077bd948aa6bef8e7e5a97c6179512f163)) + +- Update env.example ([5fb21b8](https://github.com/metaplex-foundation/aura/commit/5fb21b8f822fde53691d3fc51ebd225ad1c2d5c9)) + +- Add a separate secondary rocks backup service ([1fbfc10](https://github.com/metaplex-foundation/aura/commit/1fbfc101c7dc961397b5de73ca4c3ef54e094c22)) + +- Make the backup service a one-time job ([4d01d0c](https://github.com/metaplex-foundation/aura/commit/4d01d0c4d882b44ef136b2671cc930e61a720938)) + +- Use `PathBuf`s in backup-related functions ([221938b](https://github.com/metaplex-foundation/aura/commit/221938ba5dfb0b6f09da4b56ca33b8ec138151d7)) + +- Add metrics for `RedisReceiver` ([72a9ad0](https://github.com/metaplex-foundation/aura/commit/72a9ad0d8d1f41173e10f64e8a3357c923fdd467)) + +- Extend range of interfaces for supply object display (#382) ([f2a45d7](https://github.com/metaplex-foundation/aura/commit/f2a45d799e8b5458153fcd7308d60d965fe93935)) + +- Add Raydium price fetcher cache warmup for symbols ([39c0457](https://github.com/metaplex-foundation/aura/commit/39c0457624de16f13c709de4822ae060d4951400)) + +- Add exclusive backoff for signature fetch ([68f14ba](https://github.com/metaplex-foundation/aura/commit/68f14ba9a6298c3d958de7e0491adc94574b616f)) + +- Cover json downloader with tests ([e91de01](https://github.com/metaplex-foundation/aura/commit/e91de01b0c463ff585b1fd303d6338c2388a549f)) + +- Reverse backfill from the newest slots to the oldest (#407) ([9bda337](https://github.com/metaplex-foundation/aura/commit/9bda337e01ebb44a61bad7dee9df7b4b12b2e2dd)) + +- Rework Dockerfiles and improve build ([2324b6c](https://github.com/metaplex-foundation/aura/commit/2324b6ca06811f79161d2ceb5ea9ca87750689f7)) + +- Add docker build workflow ([4a66616](https://github.com/metaplex-foundation/aura/commit/4a66616ad5e9d5f0ae79e8237d44dda93ea1277b)) + +- Add version info to images ([9fd5ace](https://github.com/metaplex-foundation/aura/commit/9fd5ace276bda32a673f4140d2d8a97feb580533)) + +- Add repository dispatch to docker job ([5353d58](https://github.com/metaplex-foundation/aura/commit/5353d58a3de6f584d1e44d7e74ae08303370232d)) + +- Update blockbuster, bubblegum-batch-sdk to fix acct decode panic (#419) ([b01a695](https://github.com/metaplex-foundation/aura/commit/b01a69548433703e695bdc6ef526800c01227fca)) + +- Rework payload parsing (#423) ([e18e809](https://github.com/metaplex-foundation/aura/commit/e18e809fed02eaf49b35364e67612510938c8b8d)) + +- Improve env variables ([fcfadc3](https://github.com/metaplex-foundation/aura/commit/fcfadc3df60e3efae777d34fe84add45e340967b)) + +- Rework data formats used in bubblegum transaction processing (#422) ([62b6d94](https://github.com/metaplex-foundation/aura/commit/62b6d94bed0d3f1c552025ec468253b922889f49)) + +- Overhaul task spawning in the whole project (#433) ([427cc6d](https://github.com/metaplex-foundation/aura/commit/427cc6dc0c780d36b0a9a6c1c10f4bbb533dec03)) + +- Improve the deployment process (#437) ([fee9538](https://github.com/metaplex-foundation/aura/commit/fee9538befd27382a4f049fa0118b3f082350d71)) + + +### Testing + +- Add new getAsset test for regular nft collection (#377) ([daac082](https://github.com/metaplex-foundation/aura/commit/daac082cd0b16269baba2442e534cc12a3fdff00)) + +- MTG-1225 test searchAssets by owner with ShowZeroBalance: false ([c69a647](https://github.com/metaplex-foundation/aura/commit/c69a6474af1aa1c696a54bb1ca305cb71ab30659)) + +- MTG-1225 All different search query scenarios for the showZeroBalance option are covered for all token types ([ba768bd](https://github.com/metaplex-foundation/aura/commit/ba768bdb9410e2a01394ce657d82cf4b90a6ae19)) + + + ## [0.5.0] - 2025-03-04 [0.5.0]: https://github.com/metaplex-foundation/aura/compare/v0.1.0...v0.5.0