From 6837a832cf5c04f000faf1786083d3f3757e3e38 Mon Sep 17 00:00:00 2001 From: Kould Date: Wed, 23 Oct 2024 11:33:25 +0800 Subject: [PATCH 01/16] feat: add `CacheReader` for SStable --- Cargo.toml | 14 ++- rust-toolchain.toml | 2 +- src/compaction/mod.rs | 12 +- src/fs/cache_reader.rs | 81 ++++++++++++++ src/fs/mod.rs | 22 ++++ src/lib.rs | 4 +- src/ondisk/scan.rs | 7 +- src/ondisk/sstable.rs | 244 +++++++++++++++++++++++++++-------------- src/option.rs | 58 +++++----- src/stream/level.rs | 9 +- src/version/mod.rs | 20 ++-- 11 files changed, 332 insertions(+), 141 deletions(-) create mode 100644 src/fs/cache_reader.rs diff --git a/Cargo.toml b/Cargo.toml index fda0a5b1..c0970a37 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -11,7 +11,7 @@ resolver = "2" version = "0.2.0" [package.metadata] -msrv = "1.79.0" +msrv = "1.81.0" [features] bench = ["redb", "rocksdb", "sled"] @@ -49,16 +49,18 @@ path = "benches/criterion/writes.rs" required-features = ["sled"] [dependencies] +anyhow = "1" arrow = "53" async-lock = "3" async-stream = "0.3" async-trait = { version = "0.1", optional = true } -bytes = { version = "1.7", optional = true } +bytes = { version = "1.7", features = ["serde"], optional = true } crc32fast = "1" crossbeam-skiplist = "0.1" datafusion = { version = "42", optional = true } flume = { version = "0.11", features = ["async"] } -fusio = { package = "fusio", version = "0.3.1", features = [ +foyer = { version = "0.12.2" } +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio", version = "0.2.1", features = [ "aws", "dyn", "fs", @@ -66,11 +68,11 @@ fusio = { package = "fusio", version = "0.3.1", features = [ "tokio", "tokio-http", ] } -fusio-dispatch = { package = "fusio-dispatch", version = "0.2.1", features = [ +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-dispatch", version = "0.2.0", features = [ "aws", "tokio", ] } -fusio-parquet = { package = "fusio-parquet", version = "0.2.1" } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } futures-core = "0.3" futures-io = "0.3" futures-util = "0.3" @@ -82,7 +84,7 @@ regex = "1" thiserror = "1" tokio = { version = "1", features = ["io-util"], default-features = false } tokio-util = { version = "0.7" } -tonbo_macros = { version = "0.2.0", path = "tonbo_macros" } +tonbo_macros = { version = "0.1.0", path = "tonbo_macros" } tracing = "0.1" ulid = "1" diff --git a/rust-toolchain.toml b/rust-toolchain.toml index 14404d0c..db16da3a 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,3 +1,3 @@ [toolchain] -channel = "1.80.0" +channel = "1.81.0" components = ["clippy", "rust-analyzer", "rustfmt"] diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index eac33d95..11cba09c 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -10,7 +10,7 @@ use tokio::sync::oneshot; use ulid::Ulid; use crate::{ - fs::{manager::StoreManager, FileId, FileType}, + fs::{manager::StoreManager, CacheError, FileId, FileType}, inmem::{ immutable::{ArrowArrays, Builder, Immutable}, mutable::Mutable, @@ -212,15 +212,13 @@ where // This Level if level == 0 { for scope in meet_scopes_l.iter() { + let path = option.table_path(&scope.gen, level); let file = level_fs - .open_options( - &option.table_path(&scope.gen, level), - FileType::Parquet.open_options(true), - ) + .open_options(&path, FileType::Parquet.open_options(true)) .await?; streams.push(ScanStream::SsTable { - inner: SsTable::open(file) + inner: SsTable::open(option, file, path) .await? .scan( (Bound::Unbounded, Bound::Unbounded), @@ -496,6 +494,8 @@ where Fusio(#[from] fusio::Error), #[error("compaction version error: {0}")] Version(#[from] VersionError), + #[error("compaction cache error: {0}")] + Cache(#[from] CacheError), #[error("compaction channel is closed")] ChannelClose, #[error("database error: {0}")] diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs new file mode 100644 index 00000000..a700c2b9 --- /dev/null +++ b/src/fs/cache_reader.rs @@ -0,0 +1,81 @@ +use std::{ops::Range, sync::Arc}; + +use bytes::Bytes; +use foyer::{Cache, DirectFsDeviceOptions, Engine, HybridCache, HybridCacheBuilder}; +use fusio::path::{path_to_local, Path}; +use futures_core::future::BoxFuture; +use futures_util::FutureExt; +use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; + +use crate::fs::{CacheError, CacheOption}; + +pub(crate) type MetaCache = Arc>>; + +pub(crate) struct CacheReader { + inner: R, + meta_path: Path, + cache: HybridCache, Bytes>, + meta_cache: MetaCache, +} + +impl CacheReader { + pub(crate) async fn new( + option: &CacheOption, + meta_cache: MetaCache, + inner: R, + meta_path: Path, + ) -> Result, CacheError> { + let path = path_to_local(&option.path)?; + let cache: HybridCache, Bytes> = HybridCacheBuilder::new() + .memory(option.memory) + .storage(Engine::Large) // use large object disk cache engine only + .with_device_options(DirectFsDeviceOptions::new(path).with_capacity(option.local)) + .build() + .await?; + + Ok(Self { + inner, + meta_path, + cache, + meta_cache, + }) + } +} + +impl AsyncFileReader for CacheReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + async move { + if let Some(entry) = self + .cache + .get(&range) + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? + { + return Ok(entry.value().clone()); + } + + let bytes = self.inner.get_bytes(range.clone()).await?; + let entry = self.cache.insert(range, bytes); + Ok(entry.value().clone()) + } + .boxed() + } + + fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { + async move { + if let Some(entry) = self.meta_cache.get(&self.meta_path) { + return Ok(entry.value().clone()); + } + + let meta = self + .inner + .get_metadata() + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; + let entry = self.meta_cache.insert(self.meta_path.clone(), meta); + + Ok(entry.value().clone()) + } + .boxed() + } +} diff --git a/src/fs/mod.rs b/src/fs/mod.rs index 1bffbb50..aa79e977 100644 --- a/src/fs/mod.rs +++ b/src/fs/mod.rs @@ -1,11 +1,14 @@ +pub(crate) mod cache_reader; pub mod manager; use std::{ fmt::{Display, Formatter}, + io, str::FromStr, }; use fusio::{fs::OpenOptions, path::Path}; +use thiserror::Error; use ulid::{DecodeError, Ulid}; pub(crate) type FileId = Ulid; @@ -51,3 +54,22 @@ pub(crate) fn parse_file_id(path: &Path, suffix: FileType) -> Result::Ref as Encode>::Error), #[error("write recover error: {0}")] diff --git a/src/ondisk/scan.rs b/src/ondisk/scan.rs index 5a8ce66d..bf6b5ece 100644 --- a/src/ondisk/scan.rs +++ b/src/ondisk/scan.rs @@ -12,6 +12,7 @@ use parquet::arrow::{async_reader::ParquetRecordBatchStream, ProjectionMask}; use pin_project_lite::pin_project; use crate::{ + fs::cache_reader::CacheReader, record::Record, stream::record_batch::{RecordBatchEntry, RecordBatchIterator}, }; @@ -20,7 +21,7 @@ pin_project! { #[derive(Debug)] pub struct SsTableScan<'scan, R>{ #[pin] - stream: ParquetRecordBatchStream, + stream: ParquetRecordBatchStream>, iter: Option>, projection_mask: ProjectionMask, full_schema: Arc, @@ -29,8 +30,8 @@ pin_project! { } impl SsTableScan<'_, R> { - pub fn new( - stream: ParquetRecordBatchStream, + pub(crate) fn new( + stream: ParquetRecordBatchStream>, projection_mask: ProjectionMask, full_schema: Arc, ) -> Self { diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 2304d6d3..085eee00 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -1,6 +1,6 @@ use std::{marker::PhantomData, ops::Bound}; -use fusio::{dynamic::DynFile, DynRead}; +use fusio::{dynamic::DynFile, path::Path, DynRead}; use fusio_parquet::reader::AsyncReader; use futures_util::StreamExt; use parquet::arrow::{ @@ -10,16 +10,18 @@ use parquet::arrow::{ use super::{arrows::get_range_filter, scan::SsTableScan}; use crate::{ + fs::{cache_reader::CacheReader, CacheError}, record::Record, stream::record_batch::RecordBatchEntry, timestamp::{Timestamp, TimestampedRef}, + DbOption, }; pub(crate) struct SsTable where R: Record, { - reader: AsyncReader, + reader: CacheReader, _marker: PhantomData, } @@ -27,11 +29,42 @@ impl SsTable where R: Record, { - pub(crate) async fn open(file: Box) -> Result { + pub(crate) async fn open( + option: &DbOption, + file: Box, + path: Path, + ) -> Result { + let size = file.size().await?; + + Ok(SsTable { + reader: CacheReader::new( + &option.cache_option, + option.meta_cache.clone(), + AsyncReader::new(file, size).await?, + path, + ) + .await?, + _marker: PhantomData, + }) + } + + #[cfg(test)] + pub(crate) async fn open_for_test( + option: &crate::fs::CacheOption, + meta_cache: crate::fs::cache_reader::MetaCache, + file: Box, + path: Path, + ) -> Result { let size = file.size().await?; Ok(SsTable { - reader: AsyncReader::new(file, size).await?, + reader: CacheReader::new( + option, + meta_cache, + AsyncReader::new(file, size).await?, + path, + ) + .await?, _marker: PhantomData, }) } @@ -41,7 +74,7 @@ where limit: Option, projection_mask: ProjectionMask, ) -> parquet::errors::Result< - ArrowReaderBuilder>, + ArrowReaderBuilder>>, > { let mut builder = ParquetRecordBatchStreamBuilder::new_with_options( self.reader, @@ -102,6 +135,7 @@ pub(crate) mod tests { use std::{borrow::Borrow, fs::File, ops::Bound, sync::Arc}; use arrow::array::RecordBatch; + use foyer::{CacheBuilder, LruConfig}; use fusio::{dynamic::DynFile, path::Path, DynFs}; use fusio_dispatch::FsOptions; use fusio_parquet::writer::AsyncWriter; @@ -118,7 +152,7 @@ pub(crate) mod tests { use super::SsTable; use crate::{ executor::tokio::TokioExecutor, - fs::{manager::StoreManager, FileType}, + fs::{cache_reader::MetaCache, manager::StoreManager, CacheOption, FileType}, record::Record, tests::{get_test_record_batch, Test}, timestamp::Timestamped, @@ -152,15 +186,23 @@ pub(crate) mod tests { Ok(()) } - pub(crate) async fn open_sstable(store: &Arc, path: &Path) -> SsTable + pub(crate) async fn open_sstable( + option: &CacheOption, + meta_cache: MetaCache, + store: &Arc, + path: Path, + ) -> SsTable where R: Record, { - SsTable::open( + SsTable::open_for_test( + option, + meta_cache, store - .open_options(path, FileType::Parquet.open_options(true)) + .open_options(&path, FileType::Parquet.open_options(true)) .await .unwrap(), + path, ) .await .unwrap() @@ -169,6 +211,20 @@ pub(crate) mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn projection_query() { let temp_dir = tempfile::tempdir().unwrap(); + let option = CacheOption { + path: Path::from_filesystem_path(temp_dir.path()).unwrap(), + memory: 64 * 1024, + local: 256 * 1024, + }; + let meta_cache = Arc::new( + CacheBuilder::new(32) + .with_shards(4) + .with_eviction_config(LruConfig { + high_priority_pool_ratio: 0.1, + }) + .build(), + ); + let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); let record_batch = get_test_record_batch::( @@ -189,52 +245,55 @@ pub(crate) mod tests { let key = Timestamped::new("hello".to_owned(), 1.into()); { - let test_ref_1 = open_sstable::(base_fs, &table_path) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_1 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_1.get().unwrap().vstring, "hello"); assert_eq!(test_ref_1.get().unwrap().vu32, Some(12)); assert_eq!(test_ref_1.get().unwrap().vbool, None); } { - let test_ref_2 = open_sstable::(base_fs, &table_path) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_2 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_2.get().unwrap().vstring, "hello"); assert_eq!(test_ref_2.get().unwrap().vu32, None); assert_eq!(test_ref_2.get().unwrap().vbool, Some(true)); } { - let test_ref_3 = open_sstable::(base_fs, &table_path) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_3 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_3.get().unwrap().vstring, "hello"); assert_eq!(test_ref_3.get().unwrap().vu32, None); assert_eq!(test_ref_3.get().unwrap().vbool, None); @@ -246,6 +305,20 @@ pub(crate) mod tests { let temp_dir = tempfile::tempdir().unwrap(); let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); + let option = CacheOption { + path: Path::from_filesystem_path(temp_dir.path()).unwrap(), + memory: 64 * 1024 * 1024, + local: 256 * 1024 * 1024, + }; + let meta_cache = Arc::new( + CacheBuilder::new(32) + .with_shards(4) + .with_eviction_config(LruConfig { + high_priority_pool_ratio: 0.1, + }) + .build(), + ); + let record_batch = get_test_record_batch::( DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), @@ -262,19 +335,20 @@ pub(crate) mod tests { write_record_batch(file, &record_batch).await.unwrap(); { - let mut test_ref_1 = open_sstable::(base_fs, &table_path) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap(); + let mut test_ref_1 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_1.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -287,19 +361,20 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_2 = open_sstable::(base_fs, &table_path) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap(); + let mut test_ref_2 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_2.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -312,19 +387,20 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_3 = open_sstable::(base_fs, &table_path) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap(); + let mut test_ref_3 = + open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_3.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); diff --git a/src/option.rs b/src/option.rs index fd679398..d8087a36 100644 --- a/src/option.rs +++ b/src/option.rs @@ -1,8 +1,10 @@ use std::{ fmt::{Debug, Formatter}, marker::PhantomData, + sync::Arc, }; +use foyer::{CacheBuilder, LruConfig}; use fusio::path::Path; use fusio_dispatch::FsOptions; use parquet::{ @@ -13,7 +15,7 @@ use parquet::{ }; use crate::{ - fs::{FileId, FileType}, + fs::{cache_reader::MetaCache, CacheOption, FileId, FileType}, record::Record, trigger::TriggerType, version::{Version, MAX_LEVEL}, @@ -28,6 +30,7 @@ pub struct DbOption { pub(crate) clean_channel_buffer: usize, pub(crate) base_path: Path, pub(crate) base_fs: FsOptions, + pub(crate) cache_option: CacheOption, // TODO: DEBUG pub(crate) level_paths: Vec>, pub(crate) immutable_chunk_num: usize, @@ -37,6 +40,7 @@ pub struct DbOption { pub(crate) major_l_selection_table_max_num: usize, pub(crate) major_threshold_with_sst_size: usize, pub(crate) max_sst_file_size: usize, + pub(crate) meta_cache: MetaCache, pub(crate) version_log_snapshot_threshold: u32, pub(crate) trigger_type: TriggerType, pub(crate) use_wal: bool, @@ -54,6 +58,17 @@ where let (column_paths, sorting_columns) = Self::primary_key_path(primary_key_name, primary_key_index); + Self::fn_new(base_path, column_paths, sorting_columns) + } + + fn fn_new( + base_path: Path, + column_paths: ColumnPath, + sorting_columns: Vec, + ) -> Self { + let cache_path = base_path.child("cache"); + let memory = 64 * 1024 * 1024; + DbOption { immutable_chunk_num: 3, immutable_chunk_max_num: 5, @@ -79,6 +94,19 @@ where version_log_snapshot_threshold: 200, level_paths: vec![None; MAX_LEVEL], base_fs: FsOptions::Local, + cache_option: CacheOption { + path: cache_path, + memory, + local: 8 * memory, + }, + meta_cache: Arc::new( + CacheBuilder::new(32) + .with_shards(4) + .with_eviction_config(LruConfig { + high_priority_pool_ratio: 0.1, + }) + .build(), + ), } } @@ -103,32 +131,8 @@ where /// build the default configured [`DbOption`] based on the passed path fn from(base_path: Path) -> Self { let (column_paths, sorting_columns) = R::primary_key_path(); - DbOption { - immutable_chunk_num: 3, - immutable_chunk_max_num: 5, - major_threshold_with_sst_size: 4, - level_sst_magnification: 10, - max_sst_file_size: 256 * 1024 * 1024, - clean_channel_buffer: 10, - base_path, - base_fs: FsOptions::Local, - write_parquet_properties: WriterProperties::builder() - .set_compression(Compression::LZ4) - .set_column_statistics_enabled(column_paths.clone(), EnabledStatistics::Page) - .set_column_bloom_filter_enabled(column_paths.clone(), true) - .set_sorting_columns(Some(sorting_columns)) - .set_created_by(concat!("tonbo version ", env!("CARGO_PKG_VERSION")).to_owned()) - .build(), - use_wal: true, - wal_buffer_size: DEFAULT_WAL_BUFFER_SIZE, - major_default_oldest_table_num: 3, - major_l_selection_table_max_num: 4, - trigger_type: TriggerType::SizeOfMem(64 * 1024 * 1024), - _p: Default::default(), - version_log_snapshot_threshold: 200, - level_paths: vec![None; MAX_LEVEL], - } + DbOption::fn_new(base_path, column_paths, sorting_columns) } } @@ -292,6 +296,7 @@ impl Debug for DbOption { f.debug_struct("DbOption") .field("clean_channel_buffer", &self.clean_channel_buffer) .field("base_path", &self.base_path) + .field("cache_option", &self.cache_option) // TODO // .field("level_paths", &self.level_paths) .field("immutable_chunk_num", &self.immutable_chunk_num) @@ -310,6 +315,7 @@ impl Debug for DbOption { &self.major_threshold_with_sst_size, ) .field("max_sst_file_size", &self.max_sst_file_size) + .field("meta_cache", &self.meta_cache) .field( "version_log_snapshot_threshold", &self.version_log_snapshot_threshold, diff --git a/src/stream/level.rs b/src/stream/level.rs index 2039d52c..67e6905b 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -15,7 +15,7 @@ use futures_core::Stream; use parquet::{arrow::ProjectionMask, errors::ParquetError}; use crate::{ - fs::{FileId, FileType}, + fs::{CacheError, FileId, FileType}, ondisk::{scan::SsTableScan, sstable::SsTable}, record::Record, scope::Scope, @@ -32,7 +32,7 @@ where Init(FileId), Ready(SsTableScan<'level, R>), OpenFile(Pin, Error>> + 'level>>), - OpenSst(Pin, Error>> + Send + 'level>>), + OpenSst(Pin, CacheError>> + Send + 'level>>), LoadStream( Pin, ParquetError>> + Send + 'level>>, ), @@ -165,7 +165,10 @@ where }, FutureStatus::OpenFile(file_future) => match Pin::new(file_future).poll(cx) { Poll::Ready(Ok(file)) => { - self.status = FutureStatus::OpenSst(Box::pin(SsTable::open(file))); + let option = self.option.clone(); + let path = self.path.clone().unwrap(); + let future = async move { SsTable::open(&option, file, path).await }; + self.status = FutureStatus::OpenSst(Box::pin(future)); continue; } Poll::Ready(Err(err)) => { diff --git a/src/version/mod.rs b/src/version/mod.rs index 37d15c65..5f57365c 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -17,7 +17,7 @@ use thiserror::Error; use tracing::error; use crate::{ - fs::{manager::StoreManager, FileId, FileType}, + fs::{manager::StoreManager, CacheError, FileId, FileType}, ondisk::sstable::SsTable, record::Record, scope::Scope, @@ -176,14 +176,12 @@ where gen: &FileId, projection_mask: ProjectionMask, ) -> Result>, VersionError> { + let path = self.option.table_path(gen, level); let file = store - .open_options( - &self.option.table_path(gen, level), - FileType::Parquet.open_options(true), - ) + .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(file) + SsTable::::open(&self.option, file, path) .await? .get(key, projection_mask) .await @@ -218,14 +216,12 @@ where if !scope.meets_range(range) { continue; } + let path = self.option.table_path(&scope.gen, 0); let file = level_0_fs - .open_options( - &self.option.table_path(&scope.gen, 0), - FileType::Parquet.open_options(true), - ) + .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - let table = SsTable::open(file).await?; + let table = SsTable::open(&self.option, file, path).await?; streams.push(ScanStream::SsTable { inner: table @@ -318,6 +314,8 @@ where Parquet(#[from] parquet::errors::ParquetError), #[error("version fusio error: {0}")] Fusio(#[from] fusio::Error), + #[error("version cache error: {0}")] + Cache(#[from] CacheError), #[error("version ulid decode error: {0}")] UlidDecode(#[from] ulid::DecodeError), #[error("version send error: {0}")] From aa1d66a7eed2de00c0946a37fbf49d070c549f0d Mon Sep 17 00:00:00 2001 From: Kould Date: Wed, 23 Oct 2024 15:21:45 +0800 Subject: [PATCH 02/16] chore: use `CacheReader` on remotes --- src/compaction/mod.rs | 26 ++--- src/fs/cache_reader.rs | 3 +- src/fs/manager.rs | 14 ++- src/ondisk/scan.rs | 11 +- src/ondisk/sstable.rs | 239 +++++++++++++++++------------------------ src/stream/level.rs | 14 ++- src/version/cleaner.rs | 12 +-- src/version/mod.rs | 26 +++-- 8 files changed, 160 insertions(+), 185 deletions(-) diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 11cba09c..aa45d57a 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -136,7 +136,7 @@ where ) -> Result>, CompactionError> { if !batches.is_empty() { let level_0_path = option.level_fs_path(0).unwrap_or(&option.base_path); - let level_0_fs = manager.get_fs(level_0_path); + let (level_0_fs, _) = manager.get_fs(level_0_path); let mut min = None; let mut max = None; @@ -207,7 +207,7 @@ where Self::next_level_scopes(version, &mut min, &mut max, level, &meet_scopes_l)?; let level_path = option.level_fs_path(level).unwrap_or(&option.base_path); - let level_fs = manager.get_fs(level_path); + let (level_fs, is_local) = manager.get_fs(level_path); let mut streams = Vec::with_capacity(meet_scopes_l.len() + meet_scopes_ll.len()); // This Level if level == 0 { @@ -218,7 +218,7 @@ where .await?; streams.push(ScanStream::SsTable { - inner: SsTable::open(option, file, path) + inner: SsTable::open(option, file, path, is_local) .await? .scan( (Bound::Unbounded, Bound::Unbounded), @@ -240,7 +240,7 @@ where u32::MAX.into(), None, ProjectionMask::all(), - level_fs.clone(), + (level_fs.clone(), is_local), ) .ok_or(CompactionError::EmptyLevel)?; @@ -260,7 +260,7 @@ where u32::MAX.into(), None, ProjectionMask::all(), - level_fs.clone(), + (level_fs.clone(), is_local), ) .ok_or(CompactionError::EmptyLevel)?; @@ -846,14 +846,14 @@ pub(crate) mod tests { option: &Arc>, manager: &StoreManager, ) -> ((FileId, FileId, FileId, FileId, FileId), Version) { - let level_0_fs = option + let (level_0_fs, _) = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or(manager.base_fs()); - let level_1_fs = option + .unwrap_or((manager.base_fs(), true)); + let (level_1_fs, _) = option .level_fs_path(1) .map(|path| manager.get_fs(path)) - .unwrap_or(manager.base_fs()); + .unwrap_or((manager.base_fs(), true)); // level 0 let table_gen_1 = FileId::new(); @@ -1121,14 +1121,14 @@ pub(crate) mod tests { .await .unwrap(); - let level_0_fs = option + let (level_0_fs, _) = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or(manager.base_fs()); - let level_1_fs = option + .unwrap_or((manager.base_fs(), true)); + let (level_1_fs, _) = option .level_fs_path(1) .map(|path| manager.get_fs(path)) - .unwrap_or(manager.base_fs()); + .unwrap_or((manager.base_fs(), true)); let table_gen0 = FileId::new(); let table_gen1 = FileId::new(); diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs index a700c2b9..5fb25ad7 100644 --- a/src/fs/cache_reader.rs +++ b/src/fs/cache_reader.rs @@ -25,7 +25,8 @@ impl CacheReader { inner: R, meta_path: Path, ) -> Result, CacheError> { - let path = path_to_local(&option.path)?; + // SAFETY: `meta_path` must be the path of a parquet file + let path = path_to_local(&option.path.child(meta_path.filename().unwrap()))?; let cache: HybridCache, Bytes> = HybridCacheBuilder::new() .memory(option.memory) .storage(Engine::Large) // use large object disk cache engine only diff --git a/src/fs/manager.rs b/src/fs/manager.rs index 6ae83720..0be9e664 100644 --- a/src/fs/manager.rs +++ b/src/fs/manager.rs @@ -5,7 +5,7 @@ use fusio_dispatch::FsOptions; pub struct StoreManager { base_fs: Arc, - fs_map: HashMap>, + fs_map: HashMap, bool)>, } impl StoreManager { @@ -16,7 +16,10 @@ impl StoreManager { let mut fs_map = HashMap::with_capacity(levels_fs.len()); for (path, fs_options) in levels_fs.into_iter().flatten() { - fs_map.entry(path).or_insert(fs_options.parse()?); + let is_local = matches!(fs_options, FsOptions::Local); + fs_map + .entry(path) + .or_insert((fs_options.parse()?, is_local)); } let base_fs = base_options.parse()?; @@ -27,8 +30,11 @@ impl StoreManager { &self.base_fs } - pub fn get_fs(&self, path: &Path) -> &Arc { - self.fs_map.get(path).unwrap_or(&self.base_fs) + pub fn get_fs(&self, path: &Path) -> (&Arc, bool) { + self.fs_map + .get(path) + .map(|(fs, is_local)| (fs, *is_local)) + .unwrap_or((&self.base_fs, false)) } } diff --git a/src/ondisk/scan.rs b/src/ondisk/scan.rs index bf6b5ece..a7885082 100644 --- a/src/ondisk/scan.rs +++ b/src/ondisk/scan.rs @@ -6,13 +6,14 @@ use std::{ }; use arrow::datatypes::Schema; -use fusio_parquet::reader::AsyncReader; use futures_core::{ready, Stream}; -use parquet::arrow::{async_reader::ParquetRecordBatchStream, ProjectionMask}; +use parquet::arrow::{ + async_reader::{AsyncFileReader, ParquetRecordBatchStream}, + ProjectionMask, +}; use pin_project_lite::pin_project; use crate::{ - fs::cache_reader::CacheReader, record::Record, stream::record_batch::{RecordBatchEntry, RecordBatchIterator}, }; @@ -21,7 +22,7 @@ pin_project! { #[derive(Debug)] pub struct SsTableScan<'scan, R>{ #[pin] - stream: ParquetRecordBatchStream>, + stream: ParquetRecordBatchStream>, iter: Option>, projection_mask: ProjectionMask, full_schema: Arc, @@ -31,7 +32,7 @@ pin_project! { impl SsTableScan<'_, R> { pub(crate) fn new( - stream: ParquetRecordBatchStream>, + stream: ParquetRecordBatchStream>, projection_mask: ProjectionMask, full_schema: Arc, ) -> Self { diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 085eee00..14fe3299 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -5,6 +5,7 @@ use fusio_parquet::reader::AsyncReader; use futures_util::StreamExt; use parquet::arrow::{ arrow_reader::{ArrowReaderBuilder, ArrowReaderOptions}, + async_reader::AsyncFileReader, ParquetRecordBatchStreamBuilder, ProjectionMask, }; @@ -21,7 +22,7 @@ pub(crate) struct SsTable where R: Record, { - reader: CacheReader, + reader: Box, _marker: PhantomData, } @@ -33,38 +34,35 @@ where option: &DbOption, file: Box, path: Path, + is_local: bool, ) -> Result { let size = file.size().await?; + let reader = if is_local { + Box::new(AsyncReader::new(file, size).await?) as Box + } else { + Box::new( + CacheReader::new( + &option.cache_option, + option.meta_cache.clone(), + AsyncReader::new(file, size).await?, + path, + ) + .await?, + ) + }; Ok(SsTable { - reader: CacheReader::new( - &option.cache_option, - option.meta_cache.clone(), - AsyncReader::new(file, size).await?, - path, - ) - .await?, + reader, _marker: PhantomData, }) } #[cfg(test)] - pub(crate) async fn open_for_test( - option: &crate::fs::CacheOption, - meta_cache: crate::fs::cache_reader::MetaCache, - file: Box, - path: Path, - ) -> Result { + pub(crate) async fn open_local(file: Box) -> Result { let size = file.size().await?; Ok(SsTable { - reader: CacheReader::new( - option, - meta_cache, - AsyncReader::new(file, size).await?, - path, - ) - .await?, + reader: Box::new(AsyncReader::new(file, size).await?), _marker: PhantomData, }) } @@ -74,7 +72,7 @@ where limit: Option, projection_mask: ProjectionMask, ) -> parquet::errors::Result< - ArrowReaderBuilder>>, + ArrowReaderBuilder>>, > { let mut builder = ParquetRecordBatchStreamBuilder::new_with_options( self.reader, @@ -135,7 +133,6 @@ pub(crate) mod tests { use std::{borrow::Borrow, fs::File, ops::Bound, sync::Arc}; use arrow::array::RecordBatch; - use foyer::{CacheBuilder, LruConfig}; use fusio::{dynamic::DynFile, path::Path, DynFs}; use fusio_dispatch::FsOptions; use fusio_parquet::writer::AsyncWriter; @@ -152,7 +149,7 @@ pub(crate) mod tests { use super::SsTable; use crate::{ executor::tokio::TokioExecutor, - fs::{cache_reader::MetaCache, manager::StoreManager, CacheOption, FileType}, + fs::{manager::StoreManager, FileType}, record::Record, tests::{get_test_record_batch, Test}, timestamp::Timestamped, @@ -186,23 +183,15 @@ pub(crate) mod tests { Ok(()) } - pub(crate) async fn open_sstable( - option: &CacheOption, - meta_cache: MetaCache, - store: &Arc, - path: Path, - ) -> SsTable + pub(crate) async fn open_sstable(store: &Arc, path: Path) -> SsTable where R: Record, { - SsTable::open_for_test( - option, - meta_cache, + SsTable::open_local( store .open_options(&path, FileType::Parquet.open_options(true)) .await .unwrap(), - path, ) .await .unwrap() @@ -211,19 +200,6 @@ pub(crate) mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn projection_query() { let temp_dir = tempfile::tempdir().unwrap(); - let option = CacheOption { - path: Path::from_filesystem_path(temp_dir.path()).unwrap(), - memory: 64 * 1024, - local: 256 * 1024, - }; - let meta_cache = Arc::new( - CacheBuilder::new(32) - .with_shards(4) - .with_eviction_config(LruConfig { - high_priority_pool_ratio: 0.1, - }) - .build(), - ); let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); @@ -245,55 +221,52 @@ pub(crate) mod tests { let key = Timestamped::new("hello".to_owned(), 1.into()); { - let test_ref_1 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_1 = open_sstable::(base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_1.get().unwrap().vstring, "hello"); assert_eq!(test_ref_1.get().unwrap().vu32, Some(12)); assert_eq!(test_ref_1.get().unwrap().vbool, None); } { - let test_ref_2 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_2 = open_sstable::(base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_2.get().unwrap().vstring, "hello"); assert_eq!(test_ref_2.get().unwrap().vu32, None); assert_eq!(test_ref_2.get().unwrap().vbool, Some(true)); } { - let test_ref_3 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_3 = open_sstable::(base_fs, table_path.clone()) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_3.get().unwrap().vstring, "hello"); assert_eq!(test_ref_3.get().unwrap().vu32, None); assert_eq!(test_ref_3.get().unwrap().vbool, None); @@ -305,19 +278,6 @@ pub(crate) mod tests { let temp_dir = tempfile::tempdir().unwrap(); let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); - let option = CacheOption { - path: Path::from_filesystem_path(temp_dir.path()).unwrap(), - memory: 64 * 1024 * 1024, - local: 256 * 1024 * 1024, - }; - let meta_cache = Arc::new( - CacheBuilder::new(32) - .with_shards(4) - .with_eviction_config(LruConfig { - high_priority_pool_ratio: 0.1, - }) - .build(), - ); let record_batch = get_test_record_batch::( DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), @@ -335,20 +295,19 @@ pub(crate) mod tests { write_record_batch(file, &record_batch).await.unwrap(); { - let mut test_ref_1 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap(); + let mut test_ref_1 = open_sstable::(base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_1.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -361,20 +320,19 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_2 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap(); + let mut test_ref_2 = open_sstable::(base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_2.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -387,20 +345,19 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_3 = - open_sstable::(&option, meta_cache.clone(), base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap(); + let mut test_ref_3 = open_sstable::(base_fs, table_path.clone()) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_3.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); diff --git a/src/stream/level.rs b/src/stream/level.rs index 67e6905b..96027f76 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -52,6 +52,7 @@ where projection_mask: ProjectionMask, status: FutureStatus<'level, R>, fs: Arc, + is_local: bool, path: Option, } @@ -70,7 +71,7 @@ where ts: Timestamp, limit: Option, projection_mask: ProjectionMask, - fs: Arc, + (fs, is_local): (Arc, bool), ) -> Option { let (lower, upper) = range; let mut gens: VecDeque = version.level_slice[level][start..end + 1] @@ -91,6 +92,7 @@ where projection_mask, status, fs, + is_local, path: None, }) } @@ -167,7 +169,9 @@ where Poll::Ready(Ok(file)) => { let option = self.option.clone(); let path = self.path.clone().unwrap(); - let future = async move { SsTable::open(&option, file, path).await }; + let is_local = self.is_local; + let future = + async move { SsTable::open(&option, file, path, is_local).await }; self.status = FutureStatus::OpenSst(Box::pin(future)); continue; } @@ -253,7 +257,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2, 3], ), - manager.base_fs().clone(), + (manager.base_fs().clone(), true), ) .unwrap(); @@ -289,7 +293,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2, 4], ), - manager.base_fs().clone(), + (manager.base_fs().clone(), true), ) .unwrap(); @@ -325,7 +329,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2], ), - manager.base_fs().clone(), + (manager.base_fs().clone(), true), ) .unwrap(); diff --git a/src/version/cleaner.rs b/src/version/cleaner.rs index bd29e7fc..3a01d71c 100644 --- a/src/version/cleaner.rs +++ b/src/version/cleaner.rs @@ -70,21 +70,21 @@ where break; } for (gen, level) in gens { - let fs = self + let (fs, _) = self .option .level_fs_path(level) .map(|path| self.manager.get_fs(path)) - .unwrap_or(self.manager.base_fs()); + .unwrap_or((self.manager.base_fs(), true)); fs.remove(&self.option.table_path(&gen, level)).await?; } } } CleanTag::RecoverClean { wal_id: gen, level } => { - let fs = self + let (fs, _) = self .option .level_fs_path(level) .map(|path| self.manager.get_fs(path)) - .unwrap_or(self.manager.base_fs()); + .unwrap_or((self.manager.base_fs(), true)); fs.remove(&self.option.table_path(&gen, level)).await?; } } @@ -124,10 +124,10 @@ pub(crate) mod tests { let gen_1 = FileId::new(); let gen_2 = FileId::new(); let gen_3 = FileId::new(); - let fs = option + let (fs, _) = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or(manager.base_fs()); + .unwrap_or((manager.base_fs(), true)); { fs.open_options( &option.table_path(&gen_0, 0), diff --git a/src/version/mod.rs b/src/version/mod.rs index 5f57365c..ed5220da 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -125,13 +125,19 @@ where .option .level_fs_path(0) .unwrap_or(&self.option.base_path); - let level_0_fs = manager.get_fs(level_0_path); + let (level_0_fs, is_local) = manager.get_fs(level_0_path); for scope in self.level_slice[0].iter().rev() { if !scope.contains(key.value()) { continue; } if let Some(entry) = self - .table_query(level_0_fs, key, 0, &scope.gen, projection_mask.clone()) + .table_query( + (level_0_fs, is_local), + key, + 0, + &scope.gen, + projection_mask.clone(), + ) .await? { return Ok(Some(entry)); @@ -143,7 +149,7 @@ where .option .level_fs_path(leve) .unwrap_or(&self.option.base_path); - let level_fs = manager.get_fs(level_path); + let (level_fs, is_local) = manager.get_fs(level_path); if sort_runs.is_empty() { continue; } @@ -153,7 +159,7 @@ where } if let Some(entry) = self .table_query( - level_fs, + (level_fs, is_local), key, leve, &sort_runs[index].gen, @@ -170,7 +176,7 @@ where async fn table_query( &self, - store: &Arc, + (store, is_local): (&Arc, bool), key: &TimestampedRef<::Key>, level: usize, gen: &FileId, @@ -181,7 +187,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(&self.option, file, path) + SsTable::::open(&self.option, file, path, is_local) .await? .get(key, projection_mask) .await @@ -211,7 +217,7 @@ where .option .level_fs_path(0) .unwrap_or(&self.option.base_path); - let level_0_fs = manager.get_fs(level_0_path); + let (level_0_fs, is_local) = manager.get_fs(level_0_path); for scope in self.level_slice[0].iter() { if !scope.meets_range(range) { continue; @@ -221,7 +227,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - let table = SsTable::open(&self.option, file, path).await?; + let table = SsTable::open(&self.option, file, path, is_local).await?; streams.push(ScanStream::SsTable { inner: table @@ -238,7 +244,7 @@ where .option .level_fs_path(i + 1) .unwrap_or(&self.option.base_path); - let level_fs = manager.get_fs(level_path); + let (level_fs, is_local) = manager.get_fs(level_path); let (mut start, mut end) = (None, None); @@ -265,7 +271,7 @@ where ts, limit, projection_mask.clone(), - level_fs.clone(), + (level_fs.clone(), is_local), ) .unwrap(), }); From 1220569ec1c35fc6c1705cd41034cc71187b0145 Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 28 Oct 2024 17:00:35 +0800 Subject: [PATCH 03/16] refactor: `CacheReader` globally unify RangeCache to avoid duplicate creation --- Cargo.toml | 2 +- examples/declare.rs | 4 +- src/compaction/mod.rs | 22 +++- src/fs/cache_reader.rs | 213 ++++++++++++++++++++++++++++++----- src/fs/mod.rs | 7 -- src/inmem/mutable.rs | 12 +- src/lib.rs | 56 ++++++--- src/ondisk/sstable.rs | 33 +++--- src/option.rs | 62 +++++----- src/stream/level.rs | 22 ++-- src/stream/mem_projection.rs | 4 +- src/stream/merge.rs | 12 +- src/stream/package.rs | 4 +- src/transaction.rs | 48 +++++--- src/version/cleaner.rs | 8 +- src/version/mod.rs | 4 +- src/version/set.rs | 20 ++-- tests/data_integrity.rs | 4 +- 18 files changed, 385 insertions(+), 152 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index c0970a37..79020bd5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -86,7 +86,7 @@ tokio = { version = "1", features = ["io-util"], default-features = false } tokio-util = { version = "0.7" } tonbo_macros = { version = "0.1.0", path = "tonbo_macros" } tracing = "0.1" -ulid = "1" +ulid = { version = "1", features = ["serde"] } # Only used for benchmarks log = "0.4.22" diff --git a/examples/declare.rs b/examples/declare.rs index 6e6edcd3..5b569abe 100644 --- a/examples/declare.rs +++ b/examples/declare.rs @@ -22,7 +22,9 @@ async fn main() { // make sure the path exists let _ = fs::create_dir_all("./db_path/users").await; - let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); + let options = DbOption::from_path(Path::from_filesystem_path("./db_path/users").unwrap()) + .await + .unwrap(); // pluggable async runtime and I/O let db = DB::new(options, TokioExecutor::default()).await.unwrap(); diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index aa45d57a..c759e3c7 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -218,7 +218,7 @@ where .await?; streams.push(ScanStream::SsTable { - inner: SsTable::open(option, file, path, is_local) + inner: SsTable::open(option, file, scope.gen, !is_local) .await? .scan( (Bound::Unbounded, Bound::Unbounded), @@ -583,7 +583,9 @@ pub(crate) mod tests { let temp_dir = tempfile::tempdir().unwrap(); let temp_dir_l0 = tempfile::tempdir().unwrap(); - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()) + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap() .level_path( 0, Path::from_filesystem_path(temp_dir_l0.path()).unwrap(), @@ -697,7 +699,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_string(), 0, - ); + ) + .await + .unwrap(); manager .base_fs() .create_dir_all(&option.wal_dir_path()) @@ -765,7 +769,9 @@ pub(crate) mod tests { let temp_dir_l0 = TempDir::new().unwrap(); let temp_dir_l1 = TempDir::new().unwrap(); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()) + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap() .level_path( 0, Path::from_filesystem_path(temp_dir_l0.path()).unwrap(), @@ -1104,7 +1110,9 @@ pub(crate) mod tests { pub(crate) async fn major_panic() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); option.major_threshold_with_sst_size = 1; option.level_sst_magnification = 1; let manager = @@ -1211,7 +1219,9 @@ pub(crate) mod tests { async fn test_flush_major_level_sort() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 0; option.major_threshold_with_sst_size = 2; diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs index 5fb25ad7..458bb9fa 100644 --- a/src/fs/cache_reader.rs +++ b/src/fs/cache_reader.rs @@ -1,62 +1,54 @@ use std::{ops::Range, sync::Arc}; use bytes::Bytes; -use foyer::{Cache, DirectFsDeviceOptions, Engine, HybridCache, HybridCacheBuilder}; -use fusio::path::{path_to_local, Path}; +use foyer::{Cache, HybridCache}; use futures_core::future::BoxFuture; use futures_util::FutureExt; use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; -use crate::fs::{CacheError, CacheOption}; +use crate::fs::FileId; -pub(crate) type MetaCache = Arc>>; +pub(crate) type MetaCache = Arc>>; +pub(crate) type RangeCache = HybridCache<(FileId, Range), Bytes>; pub(crate) struct CacheReader { + gen: FileId, inner: R, - meta_path: Path, - cache: HybridCache, Bytes>, + range_cache: RangeCache, meta_cache: MetaCache, } impl CacheReader { - pub(crate) async fn new( - option: &CacheOption, + pub(crate) fn new( meta_cache: MetaCache, + range_cache: RangeCache, + gen: FileId, inner: R, - meta_path: Path, - ) -> Result, CacheError> { - // SAFETY: `meta_path` must be the path of a parquet file - let path = path_to_local(&option.path.child(meta_path.filename().unwrap()))?; - let cache: HybridCache, Bytes> = HybridCacheBuilder::new() - .memory(option.memory) - .storage(Engine::Large) // use large object disk cache engine only - .with_device_options(DirectFsDeviceOptions::new(path).with_capacity(option.local)) - .build() - .await?; - - Ok(Self { + ) -> CacheReader { + Self { + gen, inner, - meta_path, - cache, + range_cache, meta_cache, - }) + } } } impl AsyncFileReader for CacheReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { async move { + let key = (self.gen, range); if let Some(entry) = self - .cache - .get(&range) + .range_cache + .get(&key) .await .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? { return Ok(entry.value().clone()); } - let bytes = self.inner.get_bytes(range.clone()).await?; - let entry = self.cache.insert(range, bytes); + let bytes = self.inner.get_bytes(key.1.clone()).await?; + let entry = self.range_cache.insert(key, bytes); Ok(entry.value().clone()) } .boxed() @@ -64,7 +56,7 @@ impl AsyncFileReader for CacheReader { fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { async move { - if let Some(entry) = self.meta_cache.get(&self.meta_path) { + if let Some(entry) = self.meta_cache.get(&self.gen) { return Ok(entry.value().clone()); } @@ -73,10 +65,173 @@ impl AsyncFileReader for CacheReader { .get_metadata() .await .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; - let entry = self.meta_cache.insert(self.meta_path.clone(), meta); + let entry = self.meta_cache.insert(self.gen, meta); Ok(entry.value().clone()) } .boxed() } } + +#[cfg(test)] +pub(crate) mod tests { + use std::{ + collections::Bound, + sync::{ + atomic::{AtomicUsize, Ordering::SeqCst}, + Arc, + }, + }; + + use fusio::{dynamic::DynFile, path::Path, Error, IoBuf, IoBufMut, Read, Write}; + use futures_util::StreamExt; + use parquet::arrow::{arrow_to_parquet_schema, ProjectionMask}; + use tempfile::TempDir; + use ulid::Ulid; + + use crate::{ + compaction::tests::build_parquet_table, + fs::FileType, + ondisk::sstable::SsTable, + record::{Record, RecordInstance}, + tests::Test, + wal::log::LogType, + DbOption, + }; + + struct CountFile { + inner: Box, + read_count: Arc, + } + + impl Read for CountFile { + async fn read_exact_at(&mut self, buf: B, pos: u64) -> (Result<(), Error>, B) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_exact_at(buf, pos).await + } + + async fn read_to_end_at(&mut self, buf: Vec, pos: u64) -> (Result<(), Error>, Vec) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_to_end_at(buf, pos).await + } + + async fn size(&self) -> Result { + self.inner.size().await + } + } + + impl Write for CountFile { + async fn write_all(&mut self, buf: B) -> (Result<(), Error>, B) { + self.inner.write_all(buf).await + } + + async fn flush(&mut self) -> Result<(), Error> { + self.inner.flush().await + } + + async fn close(&mut self) -> Result<(), Error> { + self.inner.close().await + } + } + + #[tokio::test] + async fn test_cache_read() { + let temp_dir = TempDir::new().unwrap(); + let option = + DbOption::::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); + let fs = option.base_fs.clone().parse().unwrap(); + fs.create_dir_all(&option.version_log_dir_path()) + .await + .unwrap(); + fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); + + let table_gen = Ulid::new(); + build_parquet_table::( + &option, + table_gen, + vec![ + ( + LogType::Full, + Test { + vstring: 1.to_string(), + vu32: 0, + vbool: Some(true), + }, + 0.into(), + ), + ( + LogType::Full, + Test { + vstring: 2.to_string(), + vu32: 0, + vbool: Some(true), + }, + 0.into(), + ), + ( + LogType::Full, + Test { + vstring: 3.to_string(), + vu32: 0, + vbool: Some(true), + }, + 0.into(), + ), + ], + &RecordInstance::Normal, + 0, + &fs, + ) + .await + .unwrap(); + + let read_count = Arc::new(AtomicUsize::new(0)); + let table_path = option.table_path(&table_gen, 0); + for _ in 0..1000 { + let mut scan = SsTable::::open( + &option, + Box::new(CountFile { + inner: fs + .open_options(&table_path, FileType::Parquet.open_options(true)) + .await + .unwrap(), + read_count: read_count.clone(), + }), + table_gen, + true, + ) + .await + .unwrap() + .scan( + (Bound::Unbounded, Bound::Unbounded), + 0_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3, 4], + ), + ) + .await + .unwrap(); + + let entry_0 = scan.next().await.unwrap().unwrap(); + assert_eq!(entry_0.get().unwrap().vstring, "1"); + assert_eq!(entry_0.get().unwrap().vu32, Some(0)); + assert_eq!(entry_0.get().unwrap().vbool, Some(true)); + + let entry_1 = scan.next().await.unwrap().unwrap(); + assert_eq!(entry_1.get().unwrap().vstring, "2"); + assert_eq!(entry_1.get().unwrap().vu32, Some(0)); + assert_eq!(entry_1.get().unwrap().vbool, Some(true)); + + let entry_2 = scan.next().await.unwrap().unwrap(); + assert_eq!(entry_2.get().unwrap().vstring, "3"); + assert_eq!(entry_2.get().unwrap().vu32, Some(0)); + assert_eq!(entry_2.get().unwrap().vbool, Some(true)); + } + + assert_eq!(read_count.load(SeqCst), 9); + } +} diff --git a/src/fs/mod.rs b/src/fs/mod.rs index aa79e977..343ffb7f 100644 --- a/src/fs/mod.rs +++ b/src/fs/mod.rs @@ -55,13 +55,6 @@ pub(crate) fn parse_file_id(path: &Path, suffix: FileType) -> Result; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let trigger = Arc::new(TriggerFactory::create(option.trigger_type)); @@ -276,7 +278,9 @@ mod tests { async fn range() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let trigger = Arc::new(TriggerFactory::create(option.trigger_type)); @@ -363,7 +367,9 @@ mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "age".to_string(), 0, - ); + ) + .await + .unwrap(); let fs = Arc::new(TokioFs) as Arc; fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/lib.rs b/src/lib.rs index 17ac3300..d07fdc36 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -54,7 +54,7 @@ //! // make sure the path exists //! let _ = fs::create_dir_all("./db_path/users").await; //! -//! let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); +//! let options = DbOption::from_path(Path::from_filesystem_path("./db_path/users").unwrap()).await.unwrap(); //! // pluggable async runtime and I/O //! let db = DB::new(options, TokioExecutor::default()).await.unwrap(); //! // insert with owned value @@ -1513,7 +1513,9 @@ pub(crate) mod tests { let path = Path::from_filesystem_path(temp_dir.path()).unwrap(); let path_l0 = Path::from_filesystem_path(temp_dir_l0.path()).unwrap(); - let mut option = DbOption::from(path) + let mut option = DbOption::from_path(path) + .await + .unwrap() .level_path(0, path_l0, FsOptions::Local) .unwrap(); option.immutable_chunk_num = 1; @@ -1551,7 +1553,9 @@ pub(crate) mod tests { async fn test_flush() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1583,9 +1587,11 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let (task_tx, _task_rx) = bounded(1); @@ -1642,11 +1648,15 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let (desc, primary_key_index) = test_dyn_item_schema(); - let option = Arc::new(DbOption::with_path( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - "id".to_owned(), - primary_key_index, - )); + let option = Arc::new( + DbOption::with_path( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + "id".to_owned(), + primary_key_index, + ) + .await + .unwrap(), + ); manager .base_fs() .create_dir_all(&option.wal_dir_path()) @@ -1680,7 +1690,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_owned(), primary_key_index, - ); + ) + .await + .unwrap(); let db: DB = DB::with_schema(option, TokioExecutor::new(), desc, primary_key_index) .await @@ -1715,7 +1727,9 @@ pub(crate) mod tests { async fn test_get_removed() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1754,7 +1768,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_string(), primary_key_index, - ); + ) + .await + .unwrap(); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1962,7 +1978,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir1.path()).unwrap(), "id".to_string(), primary_key_index, - ); + ) + .await + .unwrap(); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1974,7 +1992,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir2.path()).unwrap(), "id".to_string(), primary_key_index, - ); + ) + .await + .unwrap(); option2.immutable_chunk_num = 1; option2.immutable_chunk_max_num = 1; option2.major_threshold_with_sst_size = 3; @@ -1986,7 +2006,9 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir3.path()).unwrap(), "id".to_string(), primary_key_index, - ); + ) + .await + .unwrap(); option3.immutable_chunk_num = 1; option3.immutable_chunk_max_num = 1; option3.major_threshold_with_sst_size = 3; diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 14fe3299..33988f9a 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -1,6 +1,6 @@ use std::{marker::PhantomData, ops::Bound}; -use fusio::{dynamic::DynFile, path::Path, DynRead}; +use fusio::{dynamic::DynFile, DynRead}; use fusio_parquet::reader::AsyncReader; use futures_util::StreamExt; use parquet::arrow::{ @@ -11,7 +11,7 @@ use parquet::arrow::{ use super::{arrows::get_range_filter, scan::SsTableScan}; use crate::{ - fs::{cache_reader::CacheReader, CacheError}, + fs::{cache_reader::CacheReader, CacheError, FileId}, record::Record, stream::record_batch::RecordBatchEntry, timestamp::{Timestamp, TimestampedRef}, @@ -33,22 +33,19 @@ where pub(crate) async fn open( option: &DbOption, file: Box, - path: Path, - is_local: bool, + gen: FileId, + enable_cache: bool, ) -> Result { let size = file.size().await?; - let reader = if is_local { + let reader = if !enable_cache { Box::new(AsyncReader::new(file, size).await?) as Box } else { - Box::new( - CacheReader::new( - &option.cache_option, - option.meta_cache.clone(), - AsyncReader::new(file, size).await?, - path, - ) - .await?, - ) + Box::new(CacheReader::new( + option.meta_cache.clone(), + option.range_cache.clone(), + gen, + AsyncReader::new(file, size).await?, + )) }; Ok(SsTable { @@ -204,7 +201,9 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); let record_batch = get_test_record_batch::( - DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), TokioExecutor::new(), ) .await; @@ -280,7 +279,9 @@ pub(crate) mod tests { let base_fs = manager.base_fs(); let record_batch = get_test_record_batch::( - DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), TokioExecutor::new(), ) .await; diff --git a/src/option.rs b/src/option.rs index d8087a36..ce844764 100644 --- a/src/option.rs +++ b/src/option.rs @@ -4,8 +4,8 @@ use std::{ sync::Arc, }; -use foyer::{CacheBuilder, LruConfig}; -use fusio::path::Path; +use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; +use fusio::path::{path_to_local, Path}; use fusio_dispatch::FsOptions; use parquet::{ basic::Compression, @@ -15,7 +15,10 @@ use parquet::{ }; use crate::{ - fs::{cache_reader::MetaCache, CacheOption, FileId, FileType}, + fs::{ + cache_reader::{MetaCache, RangeCache}, + CacheError, FileId, FileType, + }, record::Record, trigger::TriggerType, version::{Version, MAX_LEVEL}, @@ -30,7 +33,6 @@ pub struct DbOption { pub(crate) clean_channel_buffer: usize, pub(crate) base_path: Path, pub(crate) base_fs: FsOptions, - pub(crate) cache_option: CacheOption, // TODO: DEBUG pub(crate) level_paths: Vec>, pub(crate) immutable_chunk_num: usize, @@ -41,6 +43,7 @@ pub struct DbOption { pub(crate) major_threshold_with_sst_size: usize, pub(crate) max_sst_file_size: usize, pub(crate) meta_cache: MetaCache, + pub(crate) range_cache: RangeCache, pub(crate) version_log_snapshot_threshold: u32, pub(crate) trigger_type: TriggerType, pub(crate) use_wal: bool, @@ -53,23 +56,33 @@ impl DbOption where R: Record, { + pub async fn from_path(base_path: Path) -> Result> { + let (column_paths, sorting_columns) = R::primary_key_path(); + + DbOption::fn_new(base_path, column_paths, sorting_columns).await + } + /// build the default configured [`DbOption`] with base path and primary key - pub fn with_path(base_path: Path, primary_key_name: String, primary_key_index: usize) -> Self { + pub async fn with_path( + base_path: Path, + primary_key_name: String, + primary_key_index: usize, + ) -> Result> { let (column_paths, sorting_columns) = Self::primary_key_path(primary_key_name, primary_key_index); - Self::fn_new(base_path, column_paths, sorting_columns) + Self::fn_new(base_path, column_paths, sorting_columns).await } - fn fn_new( + async fn fn_new( base_path: Path, column_paths: ColumnPath, sorting_columns: Vec, - ) -> Self { + ) -> Result> { let cache_path = base_path.child("cache"); let memory = 64 * 1024 * 1024; - DbOption { + Ok(DbOption { immutable_chunk_num: 3, immutable_chunk_max_num: 5, major_threshold_with_sst_size: 4, @@ -94,11 +107,6 @@ where version_log_snapshot_threshold: 200, level_paths: vec![None; MAX_LEVEL], base_fs: FsOptions::Local, - cache_option: CacheOption { - path: cache_path, - memory, - local: 8 * memory, - }, meta_cache: Arc::new( CacheBuilder::new(32) .with_shards(4) @@ -107,7 +115,17 @@ where }) .build(), ), - } + range_cache: HybridCacheBuilder::new() + .memory(memory) + .storage(Engine::Large) // use large object disk cache engine only + .with_device_options( + DirectFsDeviceOptions::new(path_to_local(&cache_path).unwrap()) + .with_capacity(8 * memory), + ) + .build() + .await + .map_err(CacheError::from)?, + }) } fn primary_key_path( @@ -124,18 +142,6 @@ where } } -impl From for DbOption -where - R: Record, -{ - /// build the default configured [`DbOption`] based on the passed path - fn from(base_path: Path) -> Self { - let (column_paths, sorting_columns) = R::primary_key_path(); - - DbOption::fn_new(base_path, column_paths, sorting_columns) - } -} - impl DbOption where R: Record, @@ -296,7 +302,6 @@ impl Debug for DbOption { f.debug_struct("DbOption") .field("clean_channel_buffer", &self.clean_channel_buffer) .field("base_path", &self.base_path) - .field("cache_option", &self.cache_option) // TODO // .field("level_paths", &self.level_paths) .field("immutable_chunk_num", &self.immutable_chunk_num) @@ -316,6 +321,7 @@ impl Debug for DbOption { ) .field("max_sst_file_size", &self.max_sst_file_size) .field("meta_cache", &self.meta_cache) + .field("range_cache", &self.range_cache) .field( "version_log_snapshot_threshold", &self.version_log_snapshot_threshold, diff --git a/src/stream/level.rs b/src/stream/level.rs index 96027f76..8c561a45 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -53,7 +53,7 @@ where status: FutureStatus<'level, R>, fs: Arc, is_local: bool, - path: Option, + path: Option<(Path, FileId)>, } impl<'level, R> LevelStream<'level, R> @@ -109,10 +109,10 @@ where return match &mut self.status { FutureStatus::Init(gen) => { let gen = *gen; - self.path = Some(self.option.table_path(&gen, self.level)); + self.path = Some((self.option.table_path(&gen, self.level), gen)); let reader = self.fs.open_options( - self.path.as_ref().unwrap(), + &self.path.as_ref().unwrap().0, FileType::Parquet.open_options(true), ); #[allow(clippy::missing_transmute_annotations)] @@ -134,10 +134,10 @@ where Poll::Ready(None) => match self.gens.pop_front() { None => Poll::Ready(None), Some(gen) => { - self.path = Some(self.option.table_path(&gen, self.level)); + self.path = Some((self.option.table_path(&gen, self.level), gen)); let reader = self.fs.open_options( - self.path.as_ref().unwrap(), + &self.path.as_ref().unwrap().0, FileType::Parquet.open_options(true), ); #[allow(clippy::missing_transmute_annotations)] @@ -168,10 +168,10 @@ where FutureStatus::OpenFile(file_future) => match Pin::new(file_future).poll(cx) { Poll::Ready(Ok(file)) => { let option = self.option.clone(); - let path = self.path.clone().unwrap(); + let (_, gen) = self.path.clone().unwrap(); let is_local = self.is_local; let future = - async move { SsTable::open(&option, file, path, is_local).await }; + async move { SsTable::open(&option, file, gen, !is_local).await }; self.status = FutureStatus::OpenSst(Box::pin(future)); continue; } @@ -227,9 +227,11 @@ mod tests { async fn projection_scan() { let temp_dir = TempDir::new().unwrap(); let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() diff --git a/src/stream/mem_projection.rs b/src/stream/mem_projection.rs index 6671e285..a9ecef04 100644 --- a/src/stream/mem_projection.rs +++ b/src/stream/mem_projection.rs @@ -71,7 +71,9 @@ mod tests { async fn merge_mutable() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/stream/merge.rs b/src/stream/merge.rs index f58583c6..ef10949b 100644 --- a/src/stream/merge.rs +++ b/src/stream/merge.rs @@ -171,7 +171,9 @@ mod tests { async fn merge_mutable() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); @@ -265,7 +267,9 @@ mod tests { async fn merge_mutable_remove_duplicates() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); @@ -351,7 +355,9 @@ mod tests { async fn merge_mutable_limit() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/stream/package.rs b/src/stream/package.rs index c8493e8f..be4561cf 100644 --- a/src/stream/package.rs +++ b/src/stream/package.rs @@ -103,7 +103,9 @@ mod tests { async fn iter() { let temp_dir = TempDir::new().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/transaction.rs b/src/transaction.rs index 3a9d38c1..551d144b 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -277,7 +277,9 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let db = DB::::new( - DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), TokioExecutor::new(), ) .await @@ -312,9 +314,11 @@ mod tests { async fn transaction_get() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() @@ -402,7 +406,9 @@ mod tests { #[tokio::test] async fn write_conflicts() { let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); let db = DB::::new(option, TokioExecutor::new()) .await @@ -435,7 +441,9 @@ mod tests { #[tokio::test] async fn transaction_projection() { let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); let db = DB::::new(option, TokioExecutor::new()) .await @@ -463,9 +471,11 @@ mod tests { async fn transaction_scan() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() @@ -558,9 +568,11 @@ mod tests { async fn test_transaction_scan_bound() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() @@ -734,9 +746,11 @@ mod tests { async fn test_transaction_scan_limit() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() @@ -806,7 +820,9 @@ mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "age".to_string(), 0, - ); + ) + .await + .unwrap(); let db = DB::with_schema(option, TokioExecutor::default(), descs, 0) .await .unwrap(); diff --git a/src/version/cleaner.rs b/src/version/cleaner.rs index 3a01d71c..4693514c 100644 --- a/src/version/cleaner.rs +++ b/src/version/cleaner.rs @@ -116,9 +116,11 @@ pub(crate) mod tests { async fn test_cleaner() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); let gen_0 = FileId::new(); let gen_1 = FileId::new(); diff --git a/src/version/mod.rs b/src/version/mod.rs index ed5220da..10c3bbbd 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -187,7 +187,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(&self.option, file, path, is_local) + SsTable::::open(&self.option, file, *gen, !is_local) .await? .get(key, projection_mask) .await @@ -227,7 +227,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - let table = SsTable::open(&self.option, file, path, is_local).await?; + let table = SsTable::open(&self.option, file, scope.gen, !is_local).await?; streams.push(ScanStream::SsTable { inner: table diff --git a/src/version/set.rs b/src/version/set.rs index 586ca89e..d3ee1359 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -344,9 +344,11 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); let (sender, _) = bounded(1); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); manager .base_fs() .create_dir_all(&option.version_log_dir_path()) @@ -381,7 +383,9 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); let (sender, _) = bounded(1); - let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); option.version_log_snapshot_threshold = 4; let option = Arc::new(option); @@ -508,9 +512,11 @@ pub(crate) mod tests { async fn version_level_sort() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new(DbOption::from( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - )); + let option = Arc::new( + DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(), + ); let (sender, _) = bounded(1); manager diff --git a/tests/data_integrity.rs b/tests/data_integrity.rs index 239d0d83..2b8759af 100644 --- a/tests/data_integrity.rs +++ b/tests/data_integrity.rs @@ -70,7 +70,9 @@ mod tests { let mut write_hasher = crc32fast::Hasher::new(); let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) + .await + .unwrap(); let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); From bdcb2daf7b33c0fe7dbb08840a5582e8e6e4629b Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 28 Oct 2024 18:40:33 +0800 Subject: [PATCH 04/16] chore: codefmt --- examples/declare.rs | 4 +- src/compaction/mod.rs | 66 +++++++++++++++------------- src/fs/cache_reader.rs | 11 ++--- src/inmem/mutable.rs | 12 ++---- src/lib.rs | 56 ++++++++---------------- src/ondisk/sstable.rs | 21 +++++---- src/option.rs | 84 +++++++++++++++++------------------- src/stream/level.rs | 25 +++++++---- src/stream/mem_projection.rs | 4 +- src/stream/merge.rs | 12 ++---- src/stream/package.rs | 4 +- src/transaction.rs | 48 +++++++-------------- src/version/cleaner.rs | 8 ++-- src/version/mod.rs | 33 ++++++++++++-- src/version/set.rs | 68 ++++++++++++++++++++++------- tests/data_integrity.rs | 4 +- 16 files changed, 237 insertions(+), 223 deletions(-) diff --git a/examples/declare.rs b/examples/declare.rs index 5b569abe..6e6edcd3 100644 --- a/examples/declare.rs +++ b/examples/declare.rs @@ -22,9 +22,7 @@ async fn main() { // make sure the path exists let _ = fs::create_dir_all("./db_path/users").await; - let options = DbOption::from_path(Path::from_filesystem_path("./db_path/users").unwrap()) - .await - .unwrap(); + let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); // pluggable async runtime and I/O let db = DB::new(options, TokioExecutor::default()).await.unwrap(); diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index c759e3c7..3bd8b937 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -218,15 +218,21 @@ where .await?; streams.push(ScanStream::SsTable { - inner: SsTable::open(option, file, scope.gen, !is_local) - .await? - .scan( - (Bound::Unbounded, Bound::Unbounded), - u32::MAX.into(), - None, - ProjectionMask::all(), - ) - .await?, + inner: SsTable::open( + file, + scope.gen, + !is_local, + version.range_cache(), + version.meta_cache(), + ) + .await? + .scan( + (Bound::Unbounded, Bound::Unbounded), + u32::MAX.into(), + None, + ProjectionMask::all(), + ) + .await?, }); } } else { @@ -525,7 +531,7 @@ pub(crate) mod tests { tests::Test, timestamp::Timestamp, trigger::{TriggerFactory, TriggerType}, - version::{edit::VersionEdit, Version, MAX_LEVEL}, + version::{edit::VersionEdit, set::VersionSet, Version, MAX_LEVEL}, wal::log::LogType, DbError, DbOption, DB, }; @@ -583,9 +589,7 @@ pub(crate) mod tests { let temp_dir = tempfile::tempdir().unwrap(); let temp_dir_l0 = tempfile::tempdir().unwrap(); - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap() + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()) .level_path( 0, Path::from_filesystem_path(temp_dir_l0.path()).unwrap(), @@ -699,9 +703,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_string(), 0, - ) - .await - .unwrap(); + ); manager .base_fs() .create_dir_all(&option.wal_dir_path()) @@ -769,9 +771,7 @@ pub(crate) mod tests { let temp_dir_l0 = TempDir::new().unwrap(); let temp_dir_l1 = TempDir::new().unwrap(); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap() + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()) .level_path( 0, Path::from_filesystem_path(temp_dir_l0.path()).unwrap(), @@ -1061,8 +1061,14 @@ pub(crate) mod tests { .unwrap(); let (sender, _) = bounded(1); - let mut version = - Version::::new(option.clone(), sender, Arc::new(AtomicU32::default())); + let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); + let mut version = Version::::new( + option.clone(), + sender, + Arc::new(AtomicU32::default()), + range_cache, + meta_cache, + ); version.level_slice[0].push(Scope { min: 1.to_string(), max: 3.to_string(), @@ -1110,9 +1116,7 @@ pub(crate) mod tests { pub(crate) async fn major_panic() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); option.major_threshold_with_sst_size = 1; option.level_sst_magnification = 1; let manager = @@ -1181,8 +1185,14 @@ pub(crate) mod tests { let option = Arc::new(option); let (sender, _) = bounded(1); - let mut version = - Version::::new(option.clone(), sender, Arc::new(AtomicU32::default())); + let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); + let mut version = Version::::new( + option.clone(), + sender, + Arc::new(AtomicU32::default()), + range_cache, + meta_cache, + ); version.level_slice[0].push(Scope { min: 0.to_string(), max: 4.to_string(), @@ -1219,9 +1229,7 @@ pub(crate) mod tests { async fn test_flush_major_level_sort() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 0; option.major_threshold_with_sst_size = 2; diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs index 458bb9fa..3defb2d1 100644 --- a/src/fs/cache_reader.rs +++ b/src/fs/cache_reader.rs @@ -95,6 +95,7 @@ pub(crate) mod tests { ondisk::sstable::SsTable, record::{Record, RecordInstance}, tests::Test, + version::set::VersionSet, wal::log::LogType, DbOption, }; @@ -137,10 +138,7 @@ pub(crate) mod tests { #[tokio::test] async fn test_cache_read() { let temp_dir = TempDir::new().unwrap(); - let option = - DbOption::::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); let fs = option.base_fs.clone().parse().unwrap(); fs.create_dir_all(&option.version_log_dir_path()) .await @@ -189,9 +187,10 @@ pub(crate) mod tests { let read_count = Arc::new(AtomicUsize::new(0)); let table_path = option.table_path(&table_gen, 0); + let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); + for _ in 0..1000 { let mut scan = SsTable::::open( - &option, Box::new(CountFile { inner: fs .open_options(&table_path, FileType::Parquet.open_options(true)) @@ -201,6 +200,8 @@ pub(crate) mod tests { }), table_gen, true, + range_cache.clone(), + meta_cache.clone(), ) .await .unwrap() diff --git a/src/inmem/mutable.rs b/src/inmem/mutable.rs index dfdccb58..e5a1ea3c 100644 --- a/src/inmem/mutable.rs +++ b/src/inmem/mutable.rs @@ -228,9 +228,7 @@ mod tests { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let trigger = Arc::new(TriggerFactory::create(option.trigger_type)); @@ -278,9 +276,7 @@ mod tests { async fn range() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let trigger = Arc::new(TriggerFactory::create(option.trigger_type)); @@ -367,9 +363,7 @@ mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "age".to_string(), 0, - ) - .await - .unwrap(); + ); let fs = Arc::new(TokioFs) as Arc; fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/lib.rs b/src/lib.rs index d07fdc36..17ac3300 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -54,7 +54,7 @@ //! // make sure the path exists //! let _ = fs::create_dir_all("./db_path/users").await; //! -//! let options = DbOption::from_path(Path::from_filesystem_path("./db_path/users").unwrap()).await.unwrap(); +//! let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); //! // pluggable async runtime and I/O //! let db = DB::new(options, TokioExecutor::default()).await.unwrap(); //! // insert with owned value @@ -1513,9 +1513,7 @@ pub(crate) mod tests { let path = Path::from_filesystem_path(temp_dir.path()).unwrap(); let path_l0 = Path::from_filesystem_path(temp_dir_l0.path()).unwrap(); - let mut option = DbOption::from_path(path) - .await - .unwrap() + let mut option = DbOption::from(path) .level_path(0, path_l0, FsOptions::Local) .unwrap(); option.immutable_chunk_num = 1; @@ -1553,9 +1551,7 @@ pub(crate) mod tests { async fn test_flush() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1587,11 +1583,9 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); let (task_tx, _task_rx) = bounded(1); @@ -1648,15 +1642,11 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let (desc, primary_key_index) = test_dyn_item_schema(); - let option = Arc::new( - DbOption::with_path( - Path::from_filesystem_path(temp_dir.path()).unwrap(), - "id".to_owned(), - primary_key_index, - ) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::with_path( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + "id".to_owned(), + primary_key_index, + )); manager .base_fs() .create_dir_all(&option.wal_dir_path()) @@ -1690,9 +1680,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_owned(), primary_key_index, - ) - .await - .unwrap(); + ); let db: DB = DB::with_schema(option, TokioExecutor::new(), desc, primary_key_index) .await @@ -1727,9 +1715,7 @@ pub(crate) mod tests { async fn test_get_removed() { let temp_dir = TempDir::new().unwrap(); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1768,9 +1754,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "id".to_string(), primary_key_index, - ) - .await - .unwrap(); + ); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1978,9 +1962,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir1.path()).unwrap(), "id".to_string(), primary_key_index, - ) - .await - .unwrap(); + ); option.immutable_chunk_num = 1; option.immutable_chunk_max_num = 1; option.major_threshold_with_sst_size = 3; @@ -1992,9 +1974,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir2.path()).unwrap(), "id".to_string(), primary_key_index, - ) - .await - .unwrap(); + ); option2.immutable_chunk_num = 1; option2.immutable_chunk_max_num = 1; option2.major_threshold_with_sst_size = 3; @@ -2006,9 +1986,7 @@ pub(crate) mod tests { Path::from_filesystem_path(temp_dir3.path()).unwrap(), "id".to_string(), primary_key_index, - ) - .await - .unwrap(); + ); option3.immutable_chunk_num = 1; option3.immutable_chunk_max_num = 1; option3.major_threshold_with_sst_size = 3; diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 33988f9a..afb631cf 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -11,11 +11,13 @@ use parquet::arrow::{ use super::{arrows::get_range_filter, scan::SsTableScan}; use crate::{ - fs::{cache_reader::CacheReader, CacheError, FileId}, + fs::{ + cache_reader::{CacheReader, MetaCache, RangeCache}, + CacheError, FileId, + }, record::Record, stream::record_batch::RecordBatchEntry, timestamp::{Timestamp, TimestampedRef}, - DbOption, }; pub(crate) struct SsTable @@ -31,18 +33,19 @@ where R: Record, { pub(crate) async fn open( - option: &DbOption, file: Box, gen: FileId, enable_cache: bool, + range_cache: RangeCache, + meta_cache: MetaCache, ) -> Result { let size = file.size().await?; let reader = if !enable_cache { Box::new(AsyncReader::new(file, size).await?) as Box } else { Box::new(CacheReader::new( - option.meta_cache.clone(), - option.range_cache.clone(), + meta_cache, + range_cache, gen, AsyncReader::new(file, size).await?, )) @@ -201,9 +204,7 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); let record_batch = get_test_record_batch::( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), + DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), ) .await; @@ -279,9 +280,7 @@ pub(crate) mod tests { let base_fs = manager.base_fs(); let record_batch = get_test_record_batch::( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), + DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), ) .await; diff --git a/src/option.rs b/src/option.rs index ce844764..4efbcb85 100644 --- a/src/option.rs +++ b/src/option.rs @@ -1,11 +1,9 @@ use std::{ fmt::{Debug, Formatter}, marker::PhantomData, - sync::Arc, }; -use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; -use fusio::path::{path_to_local, Path}; +use fusio::path::Path; use fusio_dispatch::FsOptions; use parquet::{ basic::Compression, @@ -15,10 +13,7 @@ use parquet::{ }; use crate::{ - fs::{ - cache_reader::{MetaCache, RangeCache}, - CacheError, FileId, FileType, - }, + fs::{FileId, FileType}, record::Record, trigger::TriggerType, version::{Version, MAX_LEVEL}, @@ -30,6 +25,13 @@ const DEFAULT_WAL_BUFFER_SIZE: usize = 4 * 1024; /// configure the operating parameters of each component in the [`DB`](crate::DB) #[derive(Clone)] pub struct DbOption { + pub(crate) cache_path: Path, + pub(crate) cache_meta_capacity: usize, + pub(crate) cache_meta_shards: usize, + pub(crate) cache_meta_ratio: f64, + pub(crate) cache_range_memory: usize, + pub(crate) cache_range_disk: usize, + pub(crate) clean_channel_buffer: usize, pub(crate) base_path: Path, pub(crate) base_fs: FsOptions, @@ -42,8 +44,6 @@ pub struct DbOption { pub(crate) major_l_selection_table_max_num: usize, pub(crate) major_threshold_with_sst_size: usize, pub(crate) max_sst_file_size: usize, - pub(crate) meta_cache: MetaCache, - pub(crate) range_cache: RangeCache, pub(crate) version_log_snapshot_threshold: u32, pub(crate) trigger_type: TriggerType, pub(crate) use_wal: bool, @@ -56,33 +56,29 @@ impl DbOption where R: Record, { - pub async fn from_path(base_path: Path) -> Result> { - let (column_paths, sorting_columns) = R::primary_key_path(); - - DbOption::fn_new(base_path, column_paths, sorting_columns).await - } - /// build the default configured [`DbOption`] with base path and primary key - pub async fn with_path( - base_path: Path, - primary_key_name: String, - primary_key_index: usize, - ) -> Result> { + pub fn with_path(base_path: Path, primary_key_name: String, primary_key_index: usize) -> Self { let (column_paths, sorting_columns) = Self::primary_key_path(primary_key_name, primary_key_index); - Self::fn_new(base_path, column_paths, sorting_columns).await + Self::fn_new(base_path, column_paths, sorting_columns) } - async fn fn_new( + fn fn_new( base_path: Path, column_paths: ColumnPath, sorting_columns: Vec, - ) -> Result> { + ) -> Self { let cache_path = base_path.child("cache"); let memory = 64 * 1024 * 1024; - Ok(DbOption { + DbOption { + cache_path, + cache_meta_capacity: 32, + cache_meta_shards: 4, + cache_meta_ratio: 0.1, + cache_range_memory: memory, + cache_range_disk: 8 * memory, immutable_chunk_num: 3, immutable_chunk_max_num: 5, major_threshold_with_sst_size: 4, @@ -107,25 +103,7 @@ where version_log_snapshot_threshold: 200, level_paths: vec![None; MAX_LEVEL], base_fs: FsOptions::Local, - meta_cache: Arc::new( - CacheBuilder::new(32) - .with_shards(4) - .with_eviction_config(LruConfig { - high_priority_pool_ratio: 0.1, - }) - .build(), - ), - range_cache: HybridCacheBuilder::new() - .memory(memory) - .storage(Engine::Large) // use large object disk cache engine only - .with_device_options( - DirectFsDeviceOptions::new(path_to_local(&cache_path).unwrap()) - .with_capacity(8 * memory), - ) - .build() - .await - .map_err(CacheError::from)?, - }) + } } fn primary_key_path( @@ -142,6 +120,18 @@ where } } +impl From for DbOption +where + R: Record, +{ + /// build the default configured [`DbOption`] based on the passed path + fn from(base_path: Path) -> Self { + let (column_paths, sorting_columns) = R::primary_key_path(); + + DbOption::fn_new(base_path, column_paths, sorting_columns) + } +} + impl DbOption where R: Record, @@ -300,6 +290,12 @@ where impl Debug for DbOption { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { f.debug_struct("DbOption") + .field("cache_path", &self.cache_path) + .field("cache_meta_shards", &self.cache_meta_shards) + .field("cache_meta_capacity", &self.cache_meta_capacity) + .field("cache_meta_ratio", &self.cache_meta_ratio) + .field("cache_range_memory", &self.cache_range_memory) + .field("cache_range_disk", &self.cache_range_disk) .field("clean_channel_buffer", &self.clean_channel_buffer) .field("base_path", &self.base_path) // TODO @@ -320,8 +316,6 @@ impl Debug for DbOption { &self.major_threshold_with_sst_size, ) .field("max_sst_file_size", &self.max_sst_file_size) - .field("meta_cache", &self.meta_cache) - .field("range_cache", &self.range_cache) .field( "version_log_snapshot_threshold", &self.version_log_snapshot_threshold, diff --git a/src/stream/level.rs b/src/stream/level.rs index 8c561a45..923126bb 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -15,7 +15,10 @@ use futures_core::Stream; use parquet::{arrow::ProjectionMask, errors::ParquetError}; use crate::{ - fs::{CacheError, FileId, FileType}, + fs::{ + cache_reader::{MetaCache, RangeCache}, + CacheError, FileId, FileType, + }, ondisk::{scan::SsTableScan, sstable::SsTable}, record::Record, scope::Scope, @@ -47,6 +50,8 @@ where ts: Timestamp, level: usize, option: Arc>, + meta_cache: MetaCache, + range_cache: RangeCache, gens: VecDeque, limit: Option, projection_mask: ProjectionMask, @@ -87,6 +92,8 @@ where ts, level, option: version.option().clone(), + meta_cache: version.meta_cache(), + range_cache: version.range_cache(), gens, limit, projection_mask, @@ -167,11 +174,13 @@ where }, FutureStatus::OpenFile(file_future) => match Pin::new(file_future).poll(cx) { Poll::Ready(Ok(file)) => { - let option = self.option.clone(); + let meta_cache = self.meta_cache.clone(); + let range_cache = self.range_cache.clone(); let (_, gen) = self.path.clone().unwrap(); let is_local = self.is_local; - let future = - async move { SsTable::open(&option, file, gen, !is_local).await }; + let future = async move { + SsTable::open(file, gen, !is_local, range_cache, meta_cache).await + }; self.status = FutureStatus::OpenSst(Box::pin(future)); continue; } @@ -227,11 +236,9 @@ mod tests { async fn projection_scan() { let temp_dir = TempDir::new().unwrap(); let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() diff --git a/src/stream/mem_projection.rs b/src/stream/mem_projection.rs index a9ecef04..6671e285 100644 --- a/src/stream/mem_projection.rs +++ b/src/stream/mem_projection.rs @@ -71,9 +71,7 @@ mod tests { async fn merge_mutable() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/stream/merge.rs b/src/stream/merge.rs index ef10949b..f58583c6 100644 --- a/src/stream/merge.rs +++ b/src/stream/merge.rs @@ -171,9 +171,7 @@ mod tests { async fn merge_mutable() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); @@ -267,9 +265,7 @@ mod tests { async fn merge_mutable_remove_duplicates() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); @@ -355,9 +351,7 @@ mod tests { async fn merge_mutable_limit() { let temp_dir = tempfile::tempdir().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/stream/package.rs b/src/stream/package.rs index be4561cf..c8493e8f 100644 --- a/src/stream/package.rs +++ b/src/stream/package.rs @@ -103,9 +103,7 @@ mod tests { async fn iter() { let temp_dir = TempDir::new().unwrap(); let fs = Arc::new(TokioFs) as Arc; - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); diff --git a/src/transaction.rs b/src/transaction.rs index 551d144b..3a9d38c1 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -277,9 +277,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let db = DB::::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), + DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), ) .await @@ -314,11 +312,9 @@ mod tests { async fn transaction_get() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() @@ -406,9 +402,7 @@ mod tests { #[tokio::test] async fn write_conflicts() { let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); let db = DB::::new(option, TokioExecutor::new()) .await @@ -441,9 +435,7 @@ mod tests { #[tokio::test] async fn transaction_projection() { let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); let db = DB::::new(option, TokioExecutor::new()) .await @@ -471,11 +463,9 @@ mod tests { async fn transaction_scan() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() @@ -568,11 +558,9 @@ mod tests { async fn test_transaction_scan_bound() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() @@ -746,11 +734,9 @@ mod tests { async fn test_transaction_scan_limit() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() @@ -820,9 +806,7 @@ mod tests { Path::from_filesystem_path(temp_dir.path()).unwrap(), "age".to_string(), 0, - ) - .await - .unwrap(); + ); let db = DB::with_schema(option, TokioExecutor::default(), descs, 0) .await .unwrap(); diff --git a/src/version/cleaner.rs b/src/version/cleaner.rs index 4693514c..3a01d71c 100644 --- a/src/version/cleaner.rs +++ b/src/version/cleaner.rs @@ -116,11 +116,9 @@ pub(crate) mod tests { async fn test_cleaner() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); let gen_0 = FileId::new(); let gen_1 = FileId::new(); diff --git a/src/version/mod.rs b/src/version/mod.rs index 10c3bbbd..f061b925 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -17,7 +17,11 @@ use thiserror::Error; use tracing::error; use crate::{ - fs::{manager::StoreManager, CacheError, FileId, FileType}, + fs::{ + cache_reader::{MetaCache, RangeCache}, + manager::StoreManager, + CacheError, FileId, FileType, + }, ondisk::sstable::SsTable, record::Record, scope::Scope, @@ -49,6 +53,9 @@ where option: Arc>, timestamp: Arc, log_length: u32, + + range_cache: RangeCache, + meta_cache: MetaCache, } impl Version @@ -60,6 +67,8 @@ where option: Arc>, clean_sender: Sender, timestamp: Arc, + range_cache: RangeCache, + meta_cache: MetaCache, ) -> Self { Version { ts: Timestamp::from(0), @@ -68,12 +77,21 @@ where option: option.clone(), timestamp, log_length: 0, + range_cache, + meta_cache, } } pub(crate) fn option(&self) -> &Arc> { &self.option } + + pub(crate) fn meta_cache(&self) -> MetaCache { + self.meta_cache.clone() + } + pub(crate) fn range_cache(&self) -> RangeCache { + self.range_cache.clone() + } } impl TransactionTs for Version @@ -107,6 +125,8 @@ where option: self.option.clone(), timestamp: self.timestamp.clone(), log_length: self.log_length, + range_cache: self.range_cache.clone(), + meta_cache: self.meta_cache.clone(), } } } @@ -187,7 +207,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(&self.option, file, *gen, !is_local) + SsTable::::open(file, *gen, !is_local, self.range_cache(), self.meta_cache()) .await? .get(key, projection_mask) .await @@ -227,7 +247,14 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - let table = SsTable::open(&self.option, file, scope.gen, !is_local).await?; + let table = SsTable::open( + file, + scope.gen, + !is_local, + self.range_cache(), + self.meta_cache(), + ) + .await?; streams.push(ScanStream::SsTable { inner: table diff --git a/src/version/set.rs b/src/version/set.rs index d3ee1359..79da0b59 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -10,12 +10,17 @@ use std::{ use async_lock::RwLock; use flume::Sender; -use fusio::{dynamic::DynFile, fs::FileMeta}; +use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; +use fusio::{dynamic::DynFile, fs::FileMeta, path::path_to_local}; use futures_util::StreamExt; use super::{TransactionTs, MAX_LEVEL}; use crate::{ - fs::{manager::StoreManager, parse_file_id, FileId, FileType}, + fs::{ + cache_reader::{MetaCache, RangeCache}, + manager::StoreManager, + parse_file_id, CacheError, FileId, FileType, + }, record::Record, serdes::Encode, timestamp::Timestamp, @@ -62,6 +67,9 @@ where timestamp: Arc, option: Arc>, manager: Arc, + + range_cache: RangeCache, + meta_cache: MetaCache, } impl Clone for VersionSet @@ -75,6 +83,8 @@ where timestamp: self.timestamp.clone(), option: self.option.clone(), manager: self.manager.clone(), + range_cache: self.range_cache.clone(), + meta_cache: self.meta_cache.clone(), } } } @@ -148,6 +158,9 @@ where let timestamp = Arc::new(AtomicU32::default()); drop(log_stream); + + let (meta_cache, range_cache) = Self::build_cache(&option).await?; + let set = VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { current: Arc::new(Version:: { @@ -157,6 +170,8 @@ where option: option.clone(), timestamp: timestamp.clone(), log_length: 0, + range_cache: range_cache.clone(), + meta_cache: meta_cache.clone(), }), log_with_id: (log, log_id), })), @@ -164,12 +179,38 @@ where timestamp, option, manager, + range_cache, + meta_cache, }; set.apply_edits(edits, None, true).await?; Ok(set) } + pub(crate) async fn build_cache( + option: &DbOption, + ) -> Result<(MetaCache, RangeCache), VersionError> { + let meta_cache = Arc::new( + CacheBuilder::new(option.cache_meta_capacity) + .with_shards(option.cache_meta_shards) + .with_eviction_config(LruConfig { + high_priority_pool_ratio: option.cache_meta_ratio, + }) + .build(), + ); + let range_cache = HybridCacheBuilder::new() + .memory(option.cache_range_memory) + .storage(Engine::Large) + .with_device_options( + DirectFsDeviceOptions::new(path_to_local(&option.cache_path).unwrap()) + .with_capacity(option.cache_range_memory), + ) + .build() + .await + .map_err(CacheError::from)?; + Ok((meta_cache, range_cache)) + } + pub(crate) async fn current(&self) -> VersionRef { self.inner.read().await.current.clone() } @@ -326,6 +367,7 @@ pub(crate) mod tests { ) .await?; let timestamp = version.timestamp.clone(); + let (meta_cache, range_cache) = VersionSet::build_cache(&option).await?; Ok(VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { @@ -336,6 +378,8 @@ pub(crate) mod tests { timestamp, option, manager, + range_cache, + meta_cache, }) } @@ -344,11 +388,9 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); let (sender, _) = bounded(1); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); manager .base_fs() .create_dir_all(&option.version_log_dir_path()) @@ -383,9 +425,7 @@ pub(crate) mod tests { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); let (sender, _) = bounded(1); - let mut option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let mut option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); option.version_log_snapshot_threshold = 4; let option = Arc::new(option); @@ -512,11 +552,9 @@ pub(crate) mod tests { async fn version_level_sort() { let temp_dir = TempDir::new().unwrap(); let manager = Arc::new(StoreManager::new(FsOptions::Local, vec![]).unwrap()); - let option = Arc::new( - DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(), - ); + let option = Arc::new(DbOption::from( + Path::from_filesystem_path(temp_dir.path()).unwrap(), + )); let (sender, _) = bounded(1); manager diff --git a/tests/data_integrity.rs b/tests/data_integrity.rs index 2b8759af..239d0d83 100644 --- a/tests/data_integrity.rs +++ b/tests/data_integrity.rs @@ -70,9 +70,7 @@ mod tests { let mut write_hasher = crc32fast::Hasher::new(); let temp_dir = TempDir::new().unwrap(); - let option = DbOption::from_path(Path::from_filesystem_path(temp_dir.path()).unwrap()) - .await - .unwrap(); + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); From cc62dc011d8dc2796b0a44d6a376a5fa700aabce Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 28 Oct 2024 21:30:04 +0800 Subject: [PATCH 05/16] chore: local also uses `CacheReader` --- src/compaction/mod.rs | 25 ++++++++++++------------- src/fs/cache_reader.rs | 1 - src/fs/manager.rs | 14 ++++---------- src/ondisk/sstable.rs | 17 ++++++----------- src/stream/level.rs | 16 ++++++---------- src/version/cleaner.rs | 12 ++++++------ src/version/mod.rs | 34 +++++++++++----------------------- 7 files changed, 45 insertions(+), 74 deletions(-) diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 3bd8b937..6fa8c734 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -136,7 +136,7 @@ where ) -> Result>, CompactionError> { if !batches.is_empty() { let level_0_path = option.level_fs_path(0).unwrap_or(&option.base_path); - let (level_0_fs, _) = manager.get_fs(level_0_path); + let level_0_fs = manager.get_fs(level_0_path); let mut min = None; let mut max = None; @@ -207,7 +207,7 @@ where Self::next_level_scopes(version, &mut min, &mut max, level, &meet_scopes_l)?; let level_path = option.level_fs_path(level).unwrap_or(&option.base_path); - let (level_fs, is_local) = manager.get_fs(level_path); + let level_fs = manager.get_fs(level_path); let mut streams = Vec::with_capacity(meet_scopes_l.len() + meet_scopes_ll.len()); // This Level if level == 0 { @@ -221,7 +221,6 @@ where inner: SsTable::open( file, scope.gen, - !is_local, version.range_cache(), version.meta_cache(), ) @@ -246,7 +245,7 @@ where u32::MAX.into(), None, ProjectionMask::all(), - (level_fs.clone(), is_local), + level_fs.clone(), ) .ok_or(CompactionError::EmptyLevel)?; @@ -266,7 +265,7 @@ where u32::MAX.into(), None, ProjectionMask::all(), - (level_fs.clone(), is_local), + level_fs.clone(), ) .ok_or(CompactionError::EmptyLevel)?; @@ -852,14 +851,14 @@ pub(crate) mod tests { option: &Arc>, manager: &StoreManager, ) -> ((FileId, FileId, FileId, FileId, FileId), Version) { - let (level_0_fs, _) = option + let level_0_fs = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or((manager.base_fs(), true)); - let (level_1_fs, _) = option + .unwrap_or(manager.base_fs()); + let level_1_fs = option .level_fs_path(1) .map(|path| manager.get_fs(path)) - .unwrap_or((manager.base_fs(), true)); + .unwrap_or(manager.base_fs()); // level 0 let table_gen_1 = FileId::new(); @@ -1133,14 +1132,14 @@ pub(crate) mod tests { .await .unwrap(); - let (level_0_fs, _) = option + let level_0_fs = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or((manager.base_fs(), true)); - let (level_1_fs, _) = option + .unwrap_or(manager.base_fs()); + let level_1_fs = option .level_fs_path(1) .map(|path| manager.get_fs(path)) - .unwrap_or((manager.base_fs(), true)); + .unwrap_or(manager.base_fs()); let table_gen0 = FileId::new(); let table_gen1 = FileId::new(); diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs index 3defb2d1..0573f3c9 100644 --- a/src/fs/cache_reader.rs +++ b/src/fs/cache_reader.rs @@ -199,7 +199,6 @@ pub(crate) mod tests { read_count: read_count.clone(), }), table_gen, - true, range_cache.clone(), meta_cache.clone(), ) diff --git a/src/fs/manager.rs b/src/fs/manager.rs index 0be9e664..6ae83720 100644 --- a/src/fs/manager.rs +++ b/src/fs/manager.rs @@ -5,7 +5,7 @@ use fusio_dispatch::FsOptions; pub struct StoreManager { base_fs: Arc, - fs_map: HashMap, bool)>, + fs_map: HashMap>, } impl StoreManager { @@ -16,10 +16,7 @@ impl StoreManager { let mut fs_map = HashMap::with_capacity(levels_fs.len()); for (path, fs_options) in levels_fs.into_iter().flatten() { - let is_local = matches!(fs_options, FsOptions::Local); - fs_map - .entry(path) - .or_insert((fs_options.parse()?, is_local)); + fs_map.entry(path).or_insert(fs_options.parse()?); } let base_fs = base_options.parse()?; @@ -30,11 +27,8 @@ impl StoreManager { &self.base_fs } - pub fn get_fs(&self, path: &Path) -> (&Arc, bool) { - self.fs_map - .get(path) - .map(|(fs, is_local)| (fs, *is_local)) - .unwrap_or((&self.base_fs, false)) + pub fn get_fs(&self, path: &Path) -> &Arc { + self.fs_map.get(path).unwrap_or(&self.base_fs) } } diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index afb631cf..2672ab60 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -35,21 +35,16 @@ where pub(crate) async fn open( file: Box, gen: FileId, - enable_cache: bool, range_cache: RangeCache, meta_cache: MetaCache, ) -> Result { let size = file.size().await?; - let reader = if !enable_cache { - Box::new(AsyncReader::new(file, size).await?) as Box - } else { - Box::new(CacheReader::new( - meta_cache, - range_cache, - gen, - AsyncReader::new(file, size).await?, - )) - }; + let reader = Box::new(CacheReader::new( + meta_cache, + range_cache, + gen, + AsyncReader::new(file, size).await?, + )); Ok(SsTable { reader, diff --git a/src/stream/level.rs b/src/stream/level.rs index 923126bb..2861f81e 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -57,7 +57,6 @@ where projection_mask: ProjectionMask, status: FutureStatus<'level, R>, fs: Arc, - is_local: bool, path: Option<(Path, FileId)>, } @@ -76,7 +75,7 @@ where ts: Timestamp, limit: Option, projection_mask: ProjectionMask, - (fs, is_local): (Arc, bool), + fs: Arc, ) -> Option { let (lower, upper) = range; let mut gens: VecDeque = version.level_slice[level][start..end + 1] @@ -99,7 +98,6 @@ where projection_mask, status, fs, - is_local, path: None, }) } @@ -177,10 +175,8 @@ where let meta_cache = self.meta_cache.clone(); let range_cache = self.range_cache.clone(); let (_, gen) = self.path.clone().unwrap(); - let is_local = self.is_local; - let future = async move { - SsTable::open(file, gen, !is_local, range_cache, meta_cache).await - }; + let future = + async move { SsTable::open(file, gen, range_cache, meta_cache).await }; self.status = FutureStatus::OpenSst(Box::pin(future)); continue; } @@ -266,7 +262,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2, 3], ), - (manager.base_fs().clone(), true), + manager.base_fs().clone(), ) .unwrap(); @@ -302,7 +298,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2, 4], ), - (manager.base_fs().clone(), true), + manager.base_fs().clone(), ) .unwrap(); @@ -338,7 +334,7 @@ mod tests { &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), [0, 1, 2], ), - (manager.base_fs().clone(), true), + manager.base_fs().clone(), ) .unwrap(); diff --git a/src/version/cleaner.rs b/src/version/cleaner.rs index 3a01d71c..bd29e7fc 100644 --- a/src/version/cleaner.rs +++ b/src/version/cleaner.rs @@ -70,21 +70,21 @@ where break; } for (gen, level) in gens { - let (fs, _) = self + let fs = self .option .level_fs_path(level) .map(|path| self.manager.get_fs(path)) - .unwrap_or((self.manager.base_fs(), true)); + .unwrap_or(self.manager.base_fs()); fs.remove(&self.option.table_path(&gen, level)).await?; } } } CleanTag::RecoverClean { wal_id: gen, level } => { - let (fs, _) = self + let fs = self .option .level_fs_path(level) .map(|path| self.manager.get_fs(path)) - .unwrap_or((self.manager.base_fs(), true)); + .unwrap_or(self.manager.base_fs()); fs.remove(&self.option.table_path(&gen, level)).await?; } } @@ -124,10 +124,10 @@ pub(crate) mod tests { let gen_1 = FileId::new(); let gen_2 = FileId::new(); let gen_3 = FileId::new(); - let (fs, _) = option + let fs = option .level_fs_path(0) .map(|path| manager.get_fs(path)) - .unwrap_or((manager.base_fs(), true)); + .unwrap_or(manager.base_fs()); { fs.open_options( &option.table_path(&gen_0, 0), diff --git a/src/version/mod.rs b/src/version/mod.rs index f061b925..2510a588 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -145,19 +145,13 @@ where .option .level_fs_path(0) .unwrap_or(&self.option.base_path); - let (level_0_fs, is_local) = manager.get_fs(level_0_path); + let level_0_fs = manager.get_fs(level_0_path); for scope in self.level_slice[0].iter().rev() { if !scope.contains(key.value()) { continue; } if let Some(entry) = self - .table_query( - (level_0_fs, is_local), - key, - 0, - &scope.gen, - projection_mask.clone(), - ) + .table_query(level_0_fs, key, 0, &scope.gen, projection_mask.clone()) .await? { return Ok(Some(entry)); @@ -169,7 +163,7 @@ where .option .level_fs_path(leve) .unwrap_or(&self.option.base_path); - let (level_fs, is_local) = manager.get_fs(level_path); + let level_fs = manager.get_fs(level_path); if sort_runs.is_empty() { continue; } @@ -179,7 +173,7 @@ where } if let Some(entry) = self .table_query( - (level_fs, is_local), + level_fs, key, leve, &sort_runs[index].gen, @@ -196,7 +190,7 @@ where async fn table_query( &self, - (store, is_local): (&Arc, bool), + store: &Arc, key: &TimestampedRef<::Key>, level: usize, gen: &FileId, @@ -207,7 +201,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(file, *gen, !is_local, self.range_cache(), self.meta_cache()) + SsTable::::open(file, *gen, self.range_cache(), self.meta_cache()) .await? .get(key, projection_mask) .await @@ -237,7 +231,7 @@ where .option .level_fs_path(0) .unwrap_or(&self.option.base_path); - let (level_0_fs, is_local) = manager.get_fs(level_0_path); + let level_0_fs = manager.get_fs(level_0_path); for scope in self.level_slice[0].iter() { if !scope.meets_range(range) { continue; @@ -247,14 +241,8 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - let table = SsTable::open( - file, - scope.gen, - !is_local, - self.range_cache(), - self.meta_cache(), - ) - .await?; + let table = + SsTable::open(file, scope.gen, self.range_cache(), self.meta_cache()).await?; streams.push(ScanStream::SsTable { inner: table @@ -271,7 +259,7 @@ where .option .level_fs_path(i + 1) .unwrap_or(&self.option.base_path); - let (level_fs, is_local) = manager.get_fs(level_path); + let level_fs = manager.get_fs(level_path); let (mut start, mut end) = (None, None); @@ -298,7 +286,7 @@ where ts, limit, projection_mask.clone(), - (level_fs.clone(), is_local), + level_fs.clone(), ) .unwrap(), }); From 8f34cbebb858f270dbff3b8a964b4617d2fa13c9 Mon Sep 17 00:00:00 2001 From: Kould Date: Tue, 29 Oct 2024 16:38:52 +0800 Subject: [PATCH 06/16] chore: move cache_reader.rs to other crate --- Cargo.toml | 9 +- src/compaction/mod.rs | 33 +++- src/fs/cache_reader.rs | 237 ------------------------- src/fs/mod.rs | 15 -- src/lib.rs | 3 +- src/ondisk/sstable.rs | 10 +- src/stream/level.rs | 8 +- src/version/mod.rs | 7 +- src/version/set.rs | 55 +++--- tonbo_ext_reader/Cargo.toml | 32 ++++ tonbo_ext_reader/src/foyer_reader.rs | 253 +++++++++++++++++++++++++++ tonbo_ext_reader/src/lib.rs | 21 +++ 12 files changed, 371 insertions(+), 312 deletions(-) delete mode 100644 src/fs/cache_reader.rs create mode 100644 tonbo_ext_reader/Cargo.toml create mode 100644 tonbo_ext_reader/src/foyer_reader.rs create mode 100644 tonbo_ext_reader/src/lib.rs diff --git a/Cargo.toml b/Cargo.toml index 79020bd5..f424e6a6 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,4 +1,4 @@ -workspace = { members = ["tonbo_macros"] } +workspace = { members = [ "tonbo_ext_reader","tonbo_macros"] } [package] description = "An embedded persistent KV database in Rust." @@ -49,17 +49,15 @@ path = "benches/criterion/writes.rs" required-features = ["sled"] [dependencies] -anyhow = "1" arrow = "53" async-lock = "3" async-stream = "0.3" async-trait = { version = "0.1", optional = true } -bytes = { version = "1.7", features = ["serde"], optional = true } +bytes = { version = "1.7", optional = true } crc32fast = "1" crossbeam-skiplist = "0.1" datafusion = { version = "42", optional = true } flume = { version = "0.11", features = ["async"] } -foyer = { version = "0.12.2" } fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio", version = "0.2.1", features = [ "aws", "dyn", @@ -85,8 +83,9 @@ thiserror = "1" tokio = { version = "1", features = ["io-util"], default-features = false } tokio-util = { version = "0.7" } tonbo_macros = { version = "0.1.0", path = "tonbo_macros" } +tonbo_ext_reader = { version = "0.1.0", path = "tonbo_ext_reader" } tracing = "0.1" -ulid = { version = "1", features = ["serde"] } +ulid = "1" # Only used for benchmarks log = "0.4.22" diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 6fa8c734..ff8db6fc 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -7,10 +7,11 @@ use futures_util::StreamExt; use parquet::arrow::{AsyncArrowWriter, ProjectionMask}; use thiserror::Error; use tokio::sync::oneshot; +use tonbo_ext_reader::CacheError; use ulid::Ulid; use crate::{ - fs::{manager::StoreManager, CacheError, FileId, FileType}, + fs::{manager::StoreManager, FileId, FileType}, inmem::{ immutable::{ArrowArrays, Builder, Immutable}, mutable::Mutable, @@ -514,11 +515,15 @@ pub(crate) mod tests { use std::sync::{atomic::AtomicU32, Arc}; use flume::bounded; - use fusio::{path::Path, DynFs}; + use fusio::{ + path::{path_to_local, Path}, + DynFs, + }; use fusio_dispatch::FsOptions; use fusio_parquet::writer::AsyncWriter; use parquet::arrow::AsyncArrowWriter; use tempfile::TempDir; + use tonbo_ext_reader::foyer_reader::build_cache; use crate::{ compaction::Compactor, @@ -530,7 +535,7 @@ pub(crate) mod tests { tests::Test, timestamp::Timestamp, trigger::{TriggerFactory, TriggerType}, - version::{edit::VersionEdit, set::VersionSet, Version, MAX_LEVEL}, + version::{edit::VersionEdit, Version, MAX_LEVEL}, wal::log::LogType, DbError, DbOption, DB, }; @@ -1060,7 +1065,16 @@ pub(crate) mod tests { .unwrap(); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); + let (meta_cache, range_cache) = build_cache( + path_to_local(&option.cache_path).unwrap(), + option.cache_meta_capacity, + option.cache_meta_shards, + option.cache_meta_ratio, + option.cache_range_memory, + option.cache_range_disk, + ) + .await + .unwrap(); let mut version = Version::::new( option.clone(), sender, @@ -1184,7 +1198,16 @@ pub(crate) mod tests { let option = Arc::new(option); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); + let (meta_cache, range_cache) = build_cache( + path_to_local(&option.cache_path).unwrap(), + option.cache_meta_capacity, + option.cache_meta_shards, + option.cache_meta_ratio, + option.cache_range_memory, + option.cache_range_disk, + ) + .await + .unwrap(); let mut version = Version::::new( option.clone(), sender, diff --git a/src/fs/cache_reader.rs b/src/fs/cache_reader.rs deleted file mode 100644 index 0573f3c9..00000000 --- a/src/fs/cache_reader.rs +++ /dev/null @@ -1,237 +0,0 @@ -use std::{ops::Range, sync::Arc}; - -use bytes::Bytes; -use foyer::{Cache, HybridCache}; -use futures_core::future::BoxFuture; -use futures_util::FutureExt; -use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; - -use crate::fs::FileId; - -pub(crate) type MetaCache = Arc>>; -pub(crate) type RangeCache = HybridCache<(FileId, Range), Bytes>; - -pub(crate) struct CacheReader { - gen: FileId, - inner: R, - range_cache: RangeCache, - meta_cache: MetaCache, -} - -impl CacheReader { - pub(crate) fn new( - meta_cache: MetaCache, - range_cache: RangeCache, - gen: FileId, - inner: R, - ) -> CacheReader { - Self { - gen, - inner, - range_cache, - meta_cache, - } - } -} - -impl AsyncFileReader for CacheReader { - fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { - async move { - let key = (self.gen, range); - if let Some(entry) = self - .range_cache - .get(&key) - .await - .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? - { - return Ok(entry.value().clone()); - } - - let bytes = self.inner.get_bytes(key.1.clone()).await?; - let entry = self.range_cache.insert(key, bytes); - Ok(entry.value().clone()) - } - .boxed() - } - - fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { - async move { - if let Some(entry) = self.meta_cache.get(&self.gen) { - return Ok(entry.value().clone()); - } - - let meta = self - .inner - .get_metadata() - .await - .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; - let entry = self.meta_cache.insert(self.gen, meta); - - Ok(entry.value().clone()) - } - .boxed() - } -} - -#[cfg(test)] -pub(crate) mod tests { - use std::{ - collections::Bound, - sync::{ - atomic::{AtomicUsize, Ordering::SeqCst}, - Arc, - }, - }; - - use fusio::{dynamic::DynFile, path::Path, Error, IoBuf, IoBufMut, Read, Write}; - use futures_util::StreamExt; - use parquet::arrow::{arrow_to_parquet_schema, ProjectionMask}; - use tempfile::TempDir; - use ulid::Ulid; - - use crate::{ - compaction::tests::build_parquet_table, - fs::FileType, - ondisk::sstable::SsTable, - record::{Record, RecordInstance}, - tests::Test, - version::set::VersionSet, - wal::log::LogType, - DbOption, - }; - - struct CountFile { - inner: Box, - read_count: Arc, - } - - impl Read for CountFile { - async fn read_exact_at(&mut self, buf: B, pos: u64) -> (Result<(), Error>, B) { - self.read_count.fetch_add(1, SeqCst); - self.inner.read_exact_at(buf, pos).await - } - - async fn read_to_end_at(&mut self, buf: Vec, pos: u64) -> (Result<(), Error>, Vec) { - self.read_count.fetch_add(1, SeqCst); - self.inner.read_to_end_at(buf, pos).await - } - - async fn size(&self) -> Result { - self.inner.size().await - } - } - - impl Write for CountFile { - async fn write_all(&mut self, buf: B) -> (Result<(), Error>, B) { - self.inner.write_all(buf).await - } - - async fn flush(&mut self) -> Result<(), Error> { - self.inner.flush().await - } - - async fn close(&mut self) -> Result<(), Error> { - self.inner.close().await - } - } - - #[tokio::test] - async fn test_cache_read() { - let temp_dir = TempDir::new().unwrap(); - let option = DbOption::::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let fs = option.base_fs.clone().parse().unwrap(); - fs.create_dir_all(&option.version_log_dir_path()) - .await - .unwrap(); - fs.create_dir_all(&option.wal_dir_path()).await.unwrap(); - - let table_gen = Ulid::new(); - build_parquet_table::( - &option, - table_gen, - vec![ - ( - LogType::Full, - Test { - vstring: 1.to_string(), - vu32: 0, - vbool: Some(true), - }, - 0.into(), - ), - ( - LogType::Full, - Test { - vstring: 2.to_string(), - vu32: 0, - vbool: Some(true), - }, - 0.into(), - ), - ( - LogType::Full, - Test { - vstring: 3.to_string(), - vu32: 0, - vbool: Some(true), - }, - 0.into(), - ), - ], - &RecordInstance::Normal, - 0, - &fs, - ) - .await - .unwrap(); - - let read_count = Arc::new(AtomicUsize::new(0)); - let table_path = option.table_path(&table_gen, 0); - let (meta_cache, range_cache) = VersionSet::build_cache(&option).await.unwrap(); - - for _ in 0..1000 { - let mut scan = SsTable::::open( - Box::new(CountFile { - inner: fs - .open_options(&table_path, FileType::Parquet.open_options(true)) - .await - .unwrap(), - read_count: read_count.clone(), - }), - table_gen, - range_cache.clone(), - meta_cache.clone(), - ) - .await - .unwrap() - .scan( - (Bound::Unbounded, Bound::Unbounded), - 0_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3, 4], - ), - ) - .await - .unwrap(); - - let entry_0 = scan.next().await.unwrap().unwrap(); - assert_eq!(entry_0.get().unwrap().vstring, "1"); - assert_eq!(entry_0.get().unwrap().vu32, Some(0)); - assert_eq!(entry_0.get().unwrap().vbool, Some(true)); - - let entry_1 = scan.next().await.unwrap().unwrap(); - assert_eq!(entry_1.get().unwrap().vstring, "2"); - assert_eq!(entry_1.get().unwrap().vu32, Some(0)); - assert_eq!(entry_1.get().unwrap().vbool, Some(true)); - - let entry_2 = scan.next().await.unwrap().unwrap(); - assert_eq!(entry_2.get().unwrap().vstring, "3"); - assert_eq!(entry_2.get().unwrap().vu32, Some(0)); - assert_eq!(entry_2.get().unwrap().vbool, Some(true)); - } - - assert_eq!(read_count.load(SeqCst), 9); - } -} diff --git a/src/fs/mod.rs b/src/fs/mod.rs index 343ffb7f..1bffbb50 100644 --- a/src/fs/mod.rs +++ b/src/fs/mod.rs @@ -1,14 +1,11 @@ -pub(crate) mod cache_reader; pub mod manager; use std::{ fmt::{Display, Formatter}, - io, str::FromStr, }; use fusio::{fs::OpenOptions, path::Path}; -use thiserror::Error; use ulid::{DecodeError, Ulid}; pub(crate) type FileId = Ulid; @@ -54,15 +51,3 @@ pub(crate) fn parse_file_id(path: &Path, suffix: FileType) -> Result Result { + ) -> Result { let size = file.size().await?; let reader = Box::new(CacheReader::new( meta_cache, @@ -53,7 +51,7 @@ where } #[cfg(test)] - pub(crate) async fn open_local(file: Box) -> Result { + pub(crate) async fn open_local(file: Box) -> Result { let size = file.size().await?; Ok(SsTable { diff --git a/src/stream/level.rs b/src/stream/level.rs index 2861f81e..ebcdb688 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -13,12 +13,10 @@ use fusio::{ }; use futures_core::Stream; use parquet::{arrow::ProjectionMask, errors::ParquetError}; +use tonbo_ext_reader::{MetaCache, RangeCache}; use crate::{ - fs::{ - cache_reader::{MetaCache, RangeCache}, - CacheError, FileId, FileType, - }, + fs::{FileId, FileType}, ondisk::{scan::SsTableScan, sstable::SsTable}, record::Record, scope::Scope, @@ -35,7 +33,7 @@ where Init(FileId), Ready(SsTableScan<'level, R>), OpenFile(Pin, Error>> + 'level>>), - OpenSst(Pin, CacheError>> + Send + 'level>>), + OpenSst(Pin, Error>> + Send + 'level>>), LoadStream( Pin, ParquetError>> + Send + 'level>>, ), diff --git a/src/version/mod.rs b/src/version/mod.rs index 2510a588..b7bd9c68 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -14,14 +14,11 @@ use flume::{SendError, Sender}; use fusio::DynFs; use parquet::arrow::ProjectionMask; use thiserror::Error; +use tonbo_ext_reader::{CacheError, MetaCache, RangeCache}; use tracing::error; use crate::{ - fs::{ - cache_reader::{MetaCache, RangeCache}, - manager::StoreManager, - CacheError, FileId, FileType, - }, + fs::{manager::StoreManager, FileId, FileType}, ondisk::sstable::SsTable, record::Record, scope::Scope, diff --git a/src/version/set.rs b/src/version/set.rs index 79da0b59..8793b264 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -10,17 +10,13 @@ use std::{ use async_lock::RwLock; use flume::Sender; -use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; use fusio::{dynamic::DynFile, fs::FileMeta, path::path_to_local}; use futures_util::StreamExt; +use tonbo_ext_reader::{foyer_reader::build_cache, MetaCache, RangeCache}; use super::{TransactionTs, MAX_LEVEL}; use crate::{ - fs::{ - cache_reader::{MetaCache, RangeCache}, - manager::StoreManager, - parse_file_id, CacheError, FileId, FileType, - }, + fs::{manager::StoreManager, parse_file_id, FileId, FileType}, record::Record, serdes::Encode, timestamp::Timestamp, @@ -159,7 +155,15 @@ where let timestamp = Arc::new(AtomicU32::default()); drop(log_stream); - let (meta_cache, range_cache) = Self::build_cache(&option).await?; + let (meta_cache, range_cache) = build_cache( + path_to_local(&option.cache_path).unwrap(), + option.cache_meta_capacity, + option.cache_meta_shards, + option.cache_meta_ratio, + option.cache_range_memory, + option.cache_range_disk, + ) + .await?; let set = VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { @@ -187,30 +191,6 @@ where Ok(set) } - pub(crate) async fn build_cache( - option: &DbOption, - ) -> Result<(MetaCache, RangeCache), VersionError> { - let meta_cache = Arc::new( - CacheBuilder::new(option.cache_meta_capacity) - .with_shards(option.cache_meta_shards) - .with_eviction_config(LruConfig { - high_priority_pool_ratio: option.cache_meta_ratio, - }) - .build(), - ); - let range_cache = HybridCacheBuilder::new() - .memory(option.cache_range_memory) - .storage(Engine::Large) - .with_device_options( - DirectFsDeviceOptions::new(path_to_local(&option.cache_path).unwrap()) - .with_capacity(option.cache_range_memory), - ) - .build() - .await - .map_err(CacheError::from)?; - Ok((meta_cache, range_cache)) - } - pub(crate) async fn current(&self) -> VersionRef { self.inner.read().await.current.clone() } @@ -331,10 +311,11 @@ pub(crate) mod tests { use async_lock::RwLock; use flume::{bounded, Sender}; - use fusio::path::Path; + use fusio::path::{path_to_local, Path}; use fusio_dispatch::FsOptions; use futures_util::StreamExt; use tempfile::TempDir; + use tonbo_ext_reader::foyer_reader::build_cache; use crate::{ fs::{manager::StoreManager, FileId, FileType}, @@ -367,7 +348,15 @@ pub(crate) mod tests { ) .await?; let timestamp = version.timestamp.clone(); - let (meta_cache, range_cache) = VersionSet::build_cache(&option).await?; + let (meta_cache, range_cache) = build_cache( + path_to_local(&option.cache_path).unwrap(), + option.cache_meta_capacity, + option.cache_meta_shards, + option.cache_meta_ratio, + option.cache_range_memory, + option.cache_range_disk, + ) + .await?; Ok(VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml new file mode 100644 index 00000000..d75f344a --- /dev/null +++ b/tonbo_ext_reader/Cargo.toml @@ -0,0 +1,32 @@ +[package] +description = "ExtReader for Tonbo." +name = "tonbo_ext_reader" +version = "0.1.0" +edition = "2021" + +[dependencies] +anyhow = "1" +arrow = "53" +bytes = { version = "1.7", features = ["serde"] } +foyer = { version = "0.12" } +futures-core = "0.3" +futures-util = "0.3" +parquet = { version = "53", features = ["async"] } +thiserror = "1" +ulid = { version = "1", features = ["serde"] } + +[dev-dependencies] +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio", version = "0.2.1", features = [ + "aws", + "dyn", + "fs", + "object_store", + "tokio", + "tokio-http", +] } +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-dispatch", version = "0.2.0", features = [ + "tokio", +] } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } +tempfile = "3" +tokio = { version = "1", features = ["full"] } \ No newline at end of file diff --git a/tonbo_ext_reader/src/foyer_reader.rs b/tonbo_ext_reader/src/foyer_reader.rs new file mode 100644 index 00000000..c8b76de2 --- /dev/null +++ b/tonbo_ext_reader/src/foyer_reader.rs @@ -0,0 +1,253 @@ +use std::{ops::Range, sync::Arc}; + +use bytes::Bytes; +use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; +use futures_core::future::BoxFuture; +use futures_util::FutureExt; +use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; +use ulid::Ulid; + +use crate::{CacheError, MetaCache, RangeCache}; + +pub struct CacheReader { + gen: Ulid, + inner: R, + range_cache: RangeCache, + meta_cache: MetaCache, +} + +impl CacheReader { + pub fn new( + meta_cache: MetaCache, + range_cache: RangeCache, + gen: Ulid, + inner: R, + ) -> CacheReader { + Self { + gen, + inner, + range_cache, + meta_cache, + } + } +} + +pub async fn build_cache( + cache_path: impl AsRef, + cache_meta_capacity: usize, + cache_meta_shards: usize, + cache_meta_ratio: f64, + cache_range_memory: usize, + cache_range_disk: usize, +) -> Result<(MetaCache, RangeCache), CacheError> { + let meta_cache = Arc::new( + CacheBuilder::new(cache_meta_capacity) + .with_shards(cache_meta_shards) + .with_eviction_config(LruConfig { + high_priority_pool_ratio: cache_meta_ratio, + }) + .build(), + ); + let range_cache = HybridCacheBuilder::new() + .memory(cache_range_memory) + .storage(Engine::Large) + .with_device_options(DirectFsDeviceOptions::new(cache_path).with_capacity(cache_range_disk)) + .build() + .await + .map_err(CacheError::from)?; + Ok((meta_cache, range_cache)) +} + +impl AsyncFileReader for CacheReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + async move { + let key = (self.gen, range); + if let Some(entry) = self + .range_cache + .get(&key) + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? + { + return Ok(entry.value().clone()); + } + + let bytes = self.inner.get_bytes(key.1.clone()).await?; + let entry = self.range_cache.insert(key, bytes); + Ok(entry.value().clone()) + } + .boxed() + } + + fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { + async move { + if let Some(entry) = self.meta_cache.get(&self.gen) { + return Ok(entry.value().clone()); + } + + let meta = self + .inner + .get_metadata() + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; + let entry = self.meta_cache.insert(self.gen, meta); + + Ok(entry.value().clone()) + } + .boxed() + } +} + +#[cfg(test)] +pub(crate) mod tests { + use std::{ + fs::File, + ops::Range, + sync::{ + atomic::{AtomicUsize, Ordering::SeqCst}, + Arc, + }, + }; + + use arrow::{ + array::{BooleanArray, RecordBatch, StringArray, UInt32Array}, + datatypes::{DataType, Field, Schema}, + }; + use fusio::{ + dynamic::DynFile, fs::OpenOptions, path::Path, Error, IoBuf, IoBufMut, Read, Write, + }; + use fusio_dispatch::FsOptions; + use fusio_parquet::{reader::AsyncReader, writer::AsyncWriter}; + use parquet::arrow::{async_reader::AsyncFileReader, AsyncArrowWriter}; + use tempfile::TempDir; + use ulid::Ulid; + + use crate::foyer_reader::{build_cache, CacheReader}; + + struct CountFile { + inner: Box, + read_count: Arc, + } + + impl Read for CountFile { + async fn read_exact_at(&mut self, buf: B, pos: u64) -> (Result<(), Error>, B) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_exact_at(buf, pos).await + } + + async fn read_to_end_at(&mut self, buf: Vec, pos: u64) -> (Result<(), Error>, Vec) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_to_end_at(buf, pos).await + } + + async fn size(&self) -> Result { + self.inner.size().await + } + } + + impl Write for CountFile { + async fn write_all(&mut self, buf: B) -> (Result<(), Error>, B) { + self.inner.write_all(buf).await + } + + async fn flush(&mut self) -> Result<(), Error> { + self.inner.flush().await + } + + async fn close(&mut self) -> Result<(), Error> { + self.inner.close().await + } + } + + #[tokio::test] + async fn test_cache_read() { + let temp_dir = TempDir::new().unwrap(); + + let parquet_path = { + let path = temp_dir.path().join("test.parquet"); + let _ = File::create(&path).unwrap(); + + Path::from_filesystem_path(&path).unwrap() + }; + let fs = FsOptions::Local.parse().unwrap(); + let schema = Arc::new(Schema::new(vec![ + Field::new("_null", DataType::Boolean, false), + Field::new("_ts", DataType::UInt32, false), + Field::new("vstring", DataType::Utf8, false), + Field::new("vu32", DataType::UInt32, false), + Field::new("vbool", DataType::Boolean, true), + ])); + let mut writer = AsyncArrowWriter::try_new( + AsyncWriter::new( + fs.open_options( + &parquet_path, + OpenOptions::default().read(true).write(true).create(true), + ) + .await + .unwrap(), + ), + schema.clone(), + None, + ) + .unwrap(); + writer + .write( + &RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(BooleanArray::from(vec![false, false, false])), + Arc::new(UInt32Array::from(vec![0, 1, 2])), + Arc::new(StringArray::from(vec!["a", "b", "c"])), + Arc::new(UInt32Array::from(vec![0, 1, 2])), + Arc::new(BooleanArray::from(vec![true, true, true])), + ], + ) + .unwrap(), + ) + .await + .unwrap(); + writer.close().await.unwrap(); + + let read_count = Arc::new(AtomicUsize::new(0)); + let (meta_cache, range_cache) = build_cache( + temp_dir.path().join("cache"), + 32, + 4, + 0.1, + 64 * 1024 * 1024, + 254 * 1024 * 1024, + ) + .await + .unwrap(); + + let gen = Ulid::new(); + for _ in 0..1000 { + let file = fs + .open_options(&parquet_path, OpenOptions::default().read(true)) + .await + .unwrap(); + let content_len = file.size().await.unwrap(); + + let mut reader = CacheReader::new( + meta_cache.clone(), + range_cache.clone(), + gen, + AsyncReader::new( + Box::new(CountFile { + inner: file, + read_count: read_count.clone(), + }), + content_len, + ) + .await + .unwrap(), + ); + + let _ = AsyncFileReader::get_metadata(&mut reader).await.unwrap(); + let _ = AsyncFileReader::get_bytes(&mut reader, Range { start: 0, end: 10 }) + .await + .unwrap(); + } + + assert_eq!(read_count.load(SeqCst), 2); + } +} diff --git a/tonbo_ext_reader/src/lib.rs b/tonbo_ext_reader/src/lib.rs new file mode 100644 index 00000000..5c64128d --- /dev/null +++ b/tonbo_ext_reader/src/lib.rs @@ -0,0 +1,21 @@ +use std::{io, ops::Range, sync::Arc}; + +use bytes::Bytes; +use foyer::{Cache, HybridCache}; +use parquet::file::metadata::ParquetMetaData; +use thiserror::Error; +use ulid::Ulid; + +pub mod foyer_reader; + +// Tips: Conditional compilation adapts to different implementations +pub type MetaCache = Arc>>; +pub type RangeCache = HybridCache<(Ulid, Range), Bytes>; + +#[derive(Debug, Error)] +pub enum CacheError { + #[error("cache io error: {0}")] + Io(#[from] io::Error), + #[error("foyer error: {0}")] + Foyer(#[from] anyhow::Error), +} From c2d6893d1f40b532f56602a8a6ec5564136bcf63 Mon Sep 17 00:00:00 2001 From: Kould Date: Thu, 31 Oct 2024 17:31:50 +0800 Subject: [PATCH 07/16] chore: remove `foyer` and `tonbo` coupling --- bindings/python/Cargo.toml | 1 + bindings/python/src/db.rs | 5 +- bindings/python/src/error.rs | 1 + bindings/python/src/transaction.rs | 18 +- examples/datafusion.rs | 10 +- examples/declare.rs | 5 +- src/compaction/mod.rs | 56 +++--- src/lib.rs | 81 +++++---- src/ondisk/scan.rs | 17 +- src/ondisk/sstable.rs | 260 +++++++++++++++------------ src/option.rs | 9 +- src/stream/level.rs | 109 ++++++----- src/stream/mem_projection.rs | 20 ++- src/stream/merge.rs | 41 +++-- src/stream/mod.rs | 35 ++-- src/stream/package.rs | 17 +- src/transaction.rs | 22 ++- src/version/mod.rs | 59 ++++-- src/version/set.rs | 54 +++--- tests/data_integrity.rs | 4 +- tonbo_ext_reader/Cargo.toml | 2 +- tonbo_ext_reader/src/foyer_reader.rs | 119 +++++++----- tonbo_ext_reader/src/lib.rs | 44 ++++- 23 files changed, 593 insertions(+), 396 deletions(-) diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index 21b170ca..a8b2eddd 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -25,3 +25,4 @@ pyo3-asyncio = { package = "pyo3-asyncio-0-21", version = "0.21", features = [ ] } tokio = { version = "1", features = ["rt-multi-thread"] } tonbo = { version = "0.2.0", path = "../../" } +tonbo_ext_reader = { path = "../../tonbo_ext_reader" } diff --git a/bindings/python/src/db.rs b/bindings/python/src/db.rs index 840ff728..0af488fb 100644 --- a/bindings/python/src/db.rs +++ b/bindings/python/src/db.rs @@ -1,5 +1,4 @@ use std::sync::Arc; - use pyo3::{ prelude::*, pyclass, pymethods, @@ -12,7 +11,7 @@ use tonbo::{ record::{ColumnDesc, DynRecord}, DB, }; - +use tonbo_ext_reader::foyer_reader::FoyerReader; use crate::{ column::Column, error::{CommitError, DbError}, @@ -28,7 +27,7 @@ type PyExecutor = TokioExecutor; pub struct TonboDB { desc: Arc>, primary_key_index: usize, - db: Arc>, + db: Arc>, } #[pymethods] diff --git a/bindings/python/src/error.rs b/bindings/python/src/error.rs index 8e91373a..bd3b48f7 100644 --- a/bindings/python/src/error.rs +++ b/bindings/python/src/error.rs @@ -50,6 +50,7 @@ impl From for PyErr { tonbo::DbError::Fusio(err) => InnerError::new_err(err.to_string()), tonbo::DbError::Recover(err) => RecoverError::new_err(err.to_string()), tonbo::DbError::WalWrite(err) => PyIOError::new_err(err.to_string()), + tonbo::DbError::Cache(err) => InnerError::new_err(err.to_string()), tonbo::DbError::ExceedsMaxLevel => ExceedsMaxLevelError::new_err("Exceeds max level"), } } diff --git a/bindings/python/src/transaction.rs b/bindings/python/src/transaction.rs index 70d535d7..2c41c024 100644 --- a/bindings/python/src/transaction.rs +++ b/bindings/python/src/transaction.rs @@ -7,7 +7,7 @@ use pyo3::{ }; use pyo3_asyncio::tokio::future_into_py; use tonbo::{record::DynRecord, transaction, Projection}; - +use tonbo_ext_reader::foyer_reader::FoyerReader; use crate::{ column::Column, error::{repeated_commit_err, CommitError, DbError}, @@ -18,14 +18,14 @@ use crate::{ #[pyclass] pub struct Transaction { - txn: Option>, + txn: Option>, desc: Arc>, primary_key_index: usize, } impl Transaction { pub(crate) fn new<'txn>( - txn: transaction::Transaction<'txn, DynRecord>, + txn: transaction::Transaction<'txn, DynRecord, FoyerReader>, desc: Arc>, ) -> Self { let primary_key_index = desc @@ -37,8 +37,8 @@ impl Transaction { Transaction { txn: Some(unsafe { transmute::< - transaction::Transaction<'txn, DynRecord>, - transaction::Transaction<'static, DynRecord>, + transaction::Transaction<'txn, DynRecord, FoyerReader>, + transaction::Transaction<'static, DynRecord, FoyerReader>, >(txn) }), desc, @@ -84,8 +84,8 @@ impl Transaction { let txn = self.txn.as_ref().unwrap(); let txn = unsafe { transmute::< - &transaction::Transaction<'_, DynRecord>, - &'static transaction::Transaction<'_, DynRecord>, + &transaction::Transaction<'_, DynRecord, FoyerReader>, + &'static transaction::Transaction<'_, DynRecord, FoyerReader>, >(txn) }; @@ -169,8 +169,8 @@ impl Transaction { let txn = self.txn.as_ref().unwrap(); let txn = unsafe { transmute::< - &transaction::Transaction<'_, DynRecord>, - &'static transaction::Transaction<'_, DynRecord>, + &transaction::Transaction<'_, DynRecord, FoyerReader>, + &'static transaction::Transaction<'_, DynRecord, FoyerReader>, >(txn) }; let col_desc = self.desc.get(self.primary_key_index).unwrap(); diff --git a/examples/datafusion.rs b/examples/datafusion.rs index 2b7dd167..db571879 100644 --- a/examples/datafusion.rs +++ b/examples/datafusion.rs @@ -30,6 +30,7 @@ use tokio::fs; use tonbo::{ executor::tokio::TokioExecutor, inmem::immutable::ArrowArrays, record::Record, DbOption, DB, }; +use tonbo_ext_reader::foyer_reader::FoyerReader; use tonbo_macros::Record; #[derive(Record, Debug)] @@ -41,12 +42,12 @@ pub struct Music { } struct MusicProvider { - db: Arc>, + db: Arc>, } struct MusicExec { cache: PlanProperties, - db: Arc>, + db: Arc>, projection: Option>, limit: Option, range: (Bound<::Key>, Bound<::Key>), @@ -95,7 +96,10 @@ impl TableProvider for MusicProvider { } impl MusicExec { - fn new(db: Arc>, projection: Option<&Vec>) -> Self { + fn new( + db: Arc>, + projection: Option<&Vec>, + ) -> Self { let schema = Music::arrow_schema(); let schema = if let Some(projection) = &projection { Arc::new(schema.project(projection).unwrap()) diff --git a/examples/declare.rs b/examples/declare.rs index 6e6edcd3..0a057b46 100644 --- a/examples/declare.rs +++ b/examples/declare.rs @@ -5,6 +5,7 @@ use fusio::path::Path; use futures_util::stream::StreamExt; use tokio::fs; use tonbo::{executor::tokio::TokioExecutor, DbOption, Projection, Record, DB}; +use tonbo_ext_reader::foyer_reader::FoyerReader; /// Use macro to define schema of column family just like ORM /// It provides type-safe read & write API @@ -24,7 +25,9 @@ async fn main() { let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); // pluggable async runtime and I/O - let db = DB::new(options, TokioExecutor::default()).await.unwrap(); + let db = DB::<_, _, FoyerReader>::new(options, TokioExecutor::default()) + .await + .unwrap(); // insert with owned value db.insert(User { diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index ff8db6fc..321ede51 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -7,7 +7,7 @@ use futures_util::StreamExt; use parquet::arrow::{AsyncArrowWriter, ProjectionMask}; use thiserror::Error; use tokio::sync::oneshot; -use tonbo_ext_reader::CacheError; +use tonbo_ext_reader::{CacheError, CacheReader}; use ulid::Ulid; use crate::{ @@ -33,27 +33,29 @@ pub enum CompactTask { Flush(Option>), } -pub(crate) struct Compactor +pub(crate) struct Compactor where R: Record, + C: CacheReader, { pub(crate) option: Arc>, pub(crate) schema: Arc>>, - pub(crate) version_set: VersionSet, + pub(crate) version_set: VersionSet, pub(crate) manager: Arc, } -impl Compactor +impl Compactor where R: Record, + C: CacheReader + 'static, { pub(crate) fn new( schema: Arc>>, option: Arc>, - version_set: VersionSet, + version_set: VersionSet, manager: Arc, ) -> Self { - Compactor:: { + Compactor:: { option, schema, version_set, @@ -188,7 +190,7 @@ where #[allow(clippy::too_many_arguments)] pub(crate) async fn major_compaction( - version: &Version, + version: &Version, option: &DbOption, mut min: &R::Key, mut max: &R::Key, @@ -305,7 +307,7 @@ where } fn next_level_scopes<'a>( - version: &'a Version, + version: &'a Version, min: &mut &'a ::Key, max: &mut &'a ::Key, level: usize, @@ -328,8 +330,8 @@ where .max() .ok_or(CompactionError::EmptyLevel)?; - start_ll = Version::::scope_search(min, &version.level_slice[level + 1]); - end_ll = Version::::scope_search(max, &version.level_slice[level + 1]); + start_ll = Version::::scope_search(min, &version.level_slice[level + 1]); + end_ll = Version::::scope_search(max, &version.level_slice[level + 1]); let next_level_len = version.level_slice[level + 1].len(); for scope in version.level_slice[level + 1] @@ -345,13 +347,13 @@ where } fn this_level_scopes<'a>( - version: &'a Version, + version: &'a Version, min: &::Key, max: &::Key, level: usize, ) -> (Vec<&'a Scope<::Key>>, usize, usize) { let mut meet_scopes_l = Vec::new(); - let mut start_l = Version::::scope_search(min, &version.level_slice[level]); + let mut start_l = Version::::scope_search(min, &version.level_slice[level]); let mut end_l = start_l; let option = version.option(); @@ -386,11 +388,11 @@ where option: &DbOption, version_edits: &mut Vec::Key>>, level: usize, - streams: Vec>, + streams: Vec>, instance: &RecordInstance, fs: &Arc, ) -> Result<(), CompactionError> { - let mut stream = MergeStream::::from_vec(streams, u32::MAX.into()).await?; + let mut stream = MergeStream::::from_vec(streams, u32::MAX.into()).await?; // Kould: is the capacity parameter necessary? let mut builder = R::Columns::builder(&instance.arrow_schema::(), 8192); @@ -523,7 +525,7 @@ pub(crate) mod tests { use fusio_parquet::writer::AsyncWriter; use parquet::arrow::AsyncArrowWriter; use tempfile::TempDir; - use tonbo_ext_reader::foyer_reader::build_cache; + use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; use crate::{ compaction::Compactor, @@ -682,7 +684,7 @@ pub(crate) mod tests { .await .unwrap(); - let scope = Compactor::::minor_compaction( + let scope = Compactor::::minor_compaction( &option, None, &vec![ @@ -746,7 +748,7 @@ pub(crate) mod tests { .await .unwrap(); - let scope = Compactor::::minor_compaction( + let scope = Compactor::::minor_compaction( &option, None, &vec![ @@ -811,7 +813,7 @@ pub(crate) mod tests { let max = 5.to_string(); let mut version_edits = Vec::new(); - Compactor::::major_compaction( + Compactor::::major_compaction( &version, &option, &min, @@ -855,7 +857,10 @@ pub(crate) mod tests { pub(crate) async fn build_version( option: &Arc>, manager: &StoreManager, - ) -> ((FileId, FileId, FileId, FileId, FileId), Version) { + ) -> ( + (FileId, FileId, FileId, FileId, FileId), + Version, + ) { let level_0_fs = option .level_fs_path(0) .map(|path| manager.get_fs(path)) @@ -1065,7 +1070,7 @@ pub(crate) mod tests { .unwrap(); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = build_cache( + let (meta_cache, range_cache) = FoyerReader::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -1075,7 +1080,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let mut version = Version::::new( + let mut version = Version::::new( option.clone(), sender, Arc::new(AtomicU32::default()), @@ -1198,7 +1203,7 @@ pub(crate) mod tests { let option = Arc::new(option); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = build_cache( + let (meta_cache, range_cache) = FoyerReader::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -1208,7 +1213,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let mut version = Version::::new( + let mut version = Version::::new( option.clone(), sender, Arc::new(AtomicU32::default()), @@ -1232,7 +1237,7 @@ pub(crate) mod tests { let min = 6.to_string(); let max = 9.to_string(); - Compactor::::major_compaction( + Compactor::::major_compaction( &version, &option, &min, @@ -1261,7 +1266,8 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); + let db: DB = + DB::new(option, TokioExecutor::new()).await.unwrap(); for i in 5..9 { let item = Test { diff --git a/src/lib.rs b/src/lib.rs index 49edf830..6e7261c5 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -38,6 +38,7 @@ //! use tokio::fs; //! use tokio_util::bytes::Bytes; //! use tonbo::{executor::tokio::TokioExecutor, DbOption, Projection, Record, DB}; +//! use tonbo_ext_reader::foyer_reader::FoyerReader; //! //! // use macro to define schema of column family just like ORM //! // it provides type safety read & write API @@ -56,7 +57,8 @@ //! //! let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); //! // pluggable async runtime and I/O -//! let db = DB::new(options, TokioExecutor::default()).await.unwrap(); +//! let db: DB = +//! DB::new(options, TokioExecutor::default()).await.unwrap(); //! // insert with owned value //! db.insert(User { //! name: "Alice".into(), @@ -150,7 +152,7 @@ use record::{ColumnDesc, DynRecord, Record, RecordInstance}; use thiserror::Error; use timestamp::{Timestamp, TimestampedRef}; use tokio::sync::oneshot; -use tonbo_ext_reader::CacheError; +use tonbo_ext_reader::{CacheError, CacheReader}; pub use tonbo_macros::{KeyAttributes, Record}; use tracing::error; use transaction::{CommitError, Transaction, TransactionEntry}; @@ -171,21 +173,23 @@ use crate::{ wal::{log::LogType, RecoverError, WalFile}, }; -pub struct DB +pub struct DB where R: Record, E: Executor, + C: CacheReader, { schema: Arc>>, - version_set: VersionSet, + version_set: VersionSet, lock_map: LockMap, manager: Arc, _p: PhantomData, } -impl DB +impl DB where E: Executor + Send + Sync + 'static, + C: CacheReader + 'static, { /// Open [`DB`] with schema which determined by [`ColumnDesc`]. pub async fn with_schema( @@ -203,11 +207,12 @@ where } } -impl DB +impl DB where R: Record + Send + Sync, R::Columns: Send + Sync, E: Executor + Send + Sync + 'static, + C: CacheReader + 'static, { /// Open [`DB`] with a [`DbOption`]. This will create a new directory at the /// path specified in [`DbOption`] (if it does not exist before) and run it @@ -247,7 +252,7 @@ where let schema = Arc::new(RwLock::new( Schema::new(option.clone(), task_tx, &version_set, instance, &manager).await?, )); - let mut compactor = Compactor::::new( + let mut compactor = Compactor::::new( schema.clone(), option.clone(), version_set.clone(), @@ -287,7 +292,7 @@ where } /// open an optimistic ACID transaction - pub async fn transaction(&self) -> Transaction<'_, R> { + pub async fn transaction(&self) -> Transaction<'_, R, C> { Transaction::new( self.version_set.current().await, self.schema.read().await, @@ -447,10 +452,10 @@ impl Schema where R: Record + Send, { - async fn new( + async fn new( option: Arc>, compaction_tx: Sender, - version_set: &VersionSet, + version_set: &VersionSet, record_instance: RecordInstance, manager: &StoreManager, ) -> Result> { @@ -556,9 +561,9 @@ where self.mutable.append(None, key, ts, value).await } - async fn get<'get>( + async fn get<'get, C: CacheReader + 'static>( &'get self, - version: &'get Version, + version: &'get Version, manager: &StoreManager, key: &'get R::Key, ts: Timestamp, @@ -613,9 +618,10 @@ where } /// scan configuration intermediate structure -pub struct Scan<'scan, R> +pub struct Scan<'scan, R, C> where R: Record, + C: CacheReader, { schema: &'scan Schema, manager: &'scan StoreManager, @@ -623,27 +629,28 @@ where upper: Bound<&'scan R::Key>, ts: Timestamp, - version: &'scan Version, + version: &'scan Version, fn_pre_stream: - Box) -> Option> + Send + 'scan>, + Box) -> Option> + Send + 'scan>, limit: Option, projection_indices: Option>, projection: ProjectionMask, } -impl<'scan, R> Scan<'scan, R> +impl<'scan, R, C> Scan<'scan, R, C> where R: Record + Send, + C: CacheReader + 'static, { fn new( schema: &'scan Schema, manager: &'scan StoreManager, (lower, upper): (Bound<&'scan R::Key>, Bound<&'scan R::Key>), ts: Timestamp, - version: &'scan Version, + version: &'scan Version, fn_pre_stream: Box< - dyn FnOnce(Option) -> Option> + Send + 'scan, + dyn FnOnce(Option) -> Option> + Send + 'scan, >, ) -> Self { Self { @@ -851,6 +858,7 @@ pub(crate) mod tests { use once_cell::sync::Lazy; use parquet::{arrow::ProjectionMask, format::SortingColumn, schema::types::ColumnPath}; use tempfile::TempDir; + use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; use tracing::error; use crate::{ @@ -1089,7 +1097,7 @@ pub(crate) mod tests { option: DbOption, executor: E, ) -> RecordBatch { - let db: DB = DB::new(option.clone(), executor).await.unwrap(); + let db: DB = DB::new(option.clone(), executor).await.unwrap(); let base_fs = db.manager.base_fs(); db.write( @@ -1234,18 +1242,19 @@ pub(crate) mod tests { )) } - pub(crate) async fn build_db( + pub(crate) async fn build_db( option: Arc>, compaction_rx: Receiver, executor: E, schema: crate::Schema, - version: Version, + version: Version, manager: Arc, - ) -> Result, DbError> + ) -> Result, DbError> where R: Record + Send + Sync, R::Columns: Send + Sync, E: Executor + Send + Sync + 'static, + C: CacheReader + 'static, { { let base_fs = manager.base_fs(); @@ -1259,7 +1268,7 @@ pub(crate) mod tests { let (mut cleaner, clean_sender) = Cleaner::::new(option.clone(), manager.clone()); let version_set = build_version_set(version, clean_sender, option.clone(), manager.clone()).await?; - let mut compactor = Compactor::::new( + let mut compactor = Compactor::::new( schema.clone(), option.clone(), version_set.clone(), @@ -1525,7 +1534,8 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(/* max_mutable_len */ 5); - let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); + let db: DB = + DB::new(option, TokioExecutor::new()).await.unwrap(); for (i, item) in test_items().into_iter().enumerate() { db.write(item, 0.into()).await.unwrap(); @@ -1561,7 +1571,8 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(/* max_mutable_len */ 50); - let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); + let db: DB = + DB::new(option, TokioExecutor::new()).await.unwrap(); for item in &test_items()[0..10] { db.write(item.clone(), 0.into()).await.unwrap(); @@ -1610,9 +1621,10 @@ pub(crate) mod tests { schema.flush_wal().await.unwrap(); drop(schema); - let db: DB = DB::new(option.as_ref().to_owned(), TokioExecutor::new()) - .await - .unwrap(); + let db: DB = + DB::new(option.as_ref().to_owned(), TokioExecutor::new()) + .await + .unwrap(); let mut sort_items = BTreeMap::new(); for item in test_items() { @@ -1682,7 +1694,7 @@ pub(crate) mod tests { "id".to_owned(), primary_key_index, ); - let db: DB = + let db: DB = DB::with_schema(option, TokioExecutor::new(), desc, primary_key_index) .await .unwrap(); @@ -1722,7 +1734,8 @@ pub(crate) mod tests { option.major_threshold_with_sst_size = 3; option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); + let db: DB = + DB::new(option, TokioExecutor::new()).await.unwrap(); for (idx, item) in test_items().into_iter().enumerate() { if idx % 2 == 0 { @@ -1764,7 +1777,7 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = + let db: DB = DB::with_schema(option, TokioExecutor::new(), cols_desc, primary_key_index) .await .unwrap(); @@ -1994,7 +2007,7 @@ pub(crate) mod tests { option3.major_default_oldest_table_num = 1; option3.trigger_type = TriggerType::Length(5); - let db1: DB = DB::with_schema( + let db1: DB = DB::with_schema( option, TokioExecutor::new(), cols_desc.clone(), @@ -2002,7 +2015,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let db2: DB = DB::with_schema( + let db2: DB = DB::with_schema( option2, TokioExecutor::new(), cols_desc.clone(), @@ -2010,7 +2023,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let db3: DB = + let db3: DB = DB::with_schema(option3, TokioExecutor::new(), cols_desc, primary_key_index) .await .unwrap(); diff --git a/src/ondisk/scan.rs b/src/ondisk/scan.rs index a7885082..077f70a1 100644 --- a/src/ondisk/scan.rs +++ b/src/ondisk/scan.rs @@ -7,11 +7,9 @@ use std::{ use arrow::datatypes::Schema; use futures_core::{ready, Stream}; -use parquet::arrow::{ - async_reader::{AsyncFileReader, ParquetRecordBatchStream}, - ProjectionMask, -}; +use parquet::arrow::{async_reader::ParquetRecordBatchStream, ProjectionMask}; use pin_project_lite::pin_project; +use tonbo_ext_reader::CacheReader; use crate::{ record::Record, @@ -20,9 +18,9 @@ use crate::{ pin_project! { #[derive(Debug)] - pub struct SsTableScan<'scan, R>{ + pub struct SsTableScan<'scan, R, C>{ #[pin] - stream: ParquetRecordBatchStream>, + stream: ParquetRecordBatchStream, iter: Option>, projection_mask: ProjectionMask, full_schema: Arc, @@ -30,9 +28,9 @@ pin_project! { } } -impl SsTableScan<'_, R> { +impl SsTableScan<'_, R, C> { pub(crate) fn new( - stream: ParquetRecordBatchStream>, + stream: ParquetRecordBatchStream, projection_mask: ProjectionMask, full_schema: Arc, ) -> Self { @@ -46,9 +44,10 @@ impl SsTableScan<'_, R> { } } -impl<'scan, R> Stream for SsTableScan<'scan, R> +impl<'scan, R, C> Stream for SsTableScan<'scan, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result, parquet::errors::ParquetError>; diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 17ef1be6..5181d62b 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -5,10 +5,9 @@ use fusio_parquet::reader::AsyncReader; use futures_util::StreamExt; use parquet::arrow::{ arrow_reader::{ArrowReaderBuilder, ArrowReaderOptions}, - async_reader::AsyncFileReader, ParquetRecordBatchStreamBuilder, ProjectionMask, }; -use tonbo_ext_reader::{foyer_reader::CacheReader, MetaCache, RangeCache}; +use tonbo_ext_reader::CacheReader; use super::{arrows::get_range_filter, scan::SsTableScan}; use crate::{ @@ -18,31 +17,33 @@ use crate::{ timestamp::{Timestamp, TimestampedRef}, }; -pub(crate) struct SsTable +pub(crate) struct SsTable where R: Record, + C: CacheReader, { - reader: Box, + reader: C, _marker: PhantomData, } -impl SsTable +impl SsTable where R: Record, + C: CacheReader + 'static, { pub(crate) async fn open( file: Box, gen: FileId, - range_cache: RangeCache, - meta_cache: MetaCache, + range_cache: C::RangeCache, + meta_cache: C::MetaCache, ) -> Result { let size = file.size().await?; - let reader = Box::new(CacheReader::new( + let reader = C::new( meta_cache, range_cache, gen, AsyncReader::new(file, size).await?, - )); + ); Ok(SsTable { reader, @@ -50,23 +51,12 @@ where }) } - #[cfg(test)] - pub(crate) async fn open_local(file: Box) -> Result { - let size = file.size().await?; - - Ok(SsTable { - reader: Box::new(AsyncReader::new(file, size).await?), - _marker: PhantomData, - }) - } - async fn into_parquet_builder( self, limit: Option, projection_mask: ProjectionMask, - ) -> parquet::errors::Result< - ArrowReaderBuilder>>, - > { + ) -> parquet::errors::Result>> + { let mut builder = ParquetRecordBatchStreamBuilder::new_with_options( self.reader, ArrowReaderOptions::default().with_page_index(true), @@ -101,7 +91,7 @@ where ts: Timestamp, limit: Option, projection_mask: ProjectionMask, - ) -> Result, parquet::errors::ParquetError> { + ) -> Result, parquet::errors::ParquetError> { let builder = self .into_parquet_builder(limit, projection_mask.clone()) .await?; @@ -123,12 +113,16 @@ where #[cfg(test)] pub(crate) mod tests { - use std::{borrow::Borrow, fs::File, ops::Bound, sync::Arc}; + use std::{borrow::Borrow, fs::File, marker::PhantomData, ops::Bound, sync::Arc}; use arrow::array::RecordBatch; - use fusio::{dynamic::DynFile, path::Path, DynFs}; + use fusio::{ + dynamic::DynFile, + path::{path_to_local, Path}, + DynFs, + }; use fusio_dispatch::FsOptions; - use fusio_parquet::writer::AsyncWriter; + use fusio_parquet::{reader::AsyncReader, writer::AsyncWriter}; use futures_util::StreamExt; use parquet::{ arrow::{ @@ -138,11 +132,12 @@ pub(crate) mod tests { basic::{Compression, ZstdLevel}, file::properties::WriterProperties, }; + use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; use super::SsTable; use crate::{ executor::tokio::TokioExecutor, - fs::{manager::StoreManager, FileType}, + fs::{manager::StoreManager, FileId, FileType}, record::Record, tests::{get_test_record_batch, Test}, timestamp::Timestamped, @@ -176,18 +171,41 @@ pub(crate) mod tests { Ok(()) } - pub(crate) async fn open_sstable(store: &Arc, path: Path) -> SsTable + pub(crate) async fn open_sstable( + store: &Arc, + path: Path, + gen: FileId, + option: &DbOption, + ) -> SsTable where R: Record, + C: CacheReader, { - SsTable::open_local( - store - .open_options(&path, FileType::Parquet.open_options(true)) - .await - .unwrap(), + let file = store + .open_options(&path, FileType::Parquet.open_options(true)) + .await + .unwrap(); + let size = file.size().await.unwrap(); + let (meta_cache, range_cache) = C::build_caches( + path_to_local(&option.cache_path).unwrap(), + option.cache_meta_capacity, + option.cache_meta_shards, + option.cache_meta_ratio, + option.cache_range_memory, + option.cache_range_disk, ) .await - .unwrap() + .unwrap(); + + SsTable { + reader: C::new( + meta_cache, + range_cache, + gen, + AsyncReader::new(file, size).await.unwrap(), + ), + _marker: PhantomData, + } } #[tokio::test(flavor = "multi_thread", worker_threads = 2)] @@ -196,14 +214,13 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); - let record_batch = get_test_record_batch::( - DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), - TokioExecutor::new(), - ) - .await; + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let record_batch = + get_test_record_batch::(option.clone(), TokioExecutor::new()).await; let table_path = temp_dir.path().join("projection_query_test.parquet"); let _ = File::create(&table_path).unwrap(); let table_path = Path::from_filesystem_path(table_path).unwrap(); + let table_gen = FileId::new(); let file = base_fs .open_options(&table_path, FileType::Parquet.open_options(false)) @@ -214,52 +231,55 @@ pub(crate) mod tests { let key = Timestamped::new("hello".to_owned(), 1.into()); { - let test_ref_1 = open_sstable::(base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_1 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_1.get().unwrap().vstring, "hello"); assert_eq!(test_ref_1.get().unwrap().vu32, Some(12)); assert_eq!(test_ref_1.get().unwrap().vbool, None); } { - let test_ref_2 = open_sstable::(base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_2 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_2.get().unwrap().vstring, "hello"); assert_eq!(test_ref_2.get().unwrap().vu32, None); assert_eq!(test_ref_2.get().unwrap().vbool, Some(true)); } { - let test_ref_3 = open_sstable::(base_fs, table_path.clone()) - .await - .get( - key.borrow(), - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap() - .unwrap(); + let test_ref_3 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .get( + key.borrow(), + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap() + .unwrap(); assert_eq!(test_ref_3.get().unwrap().vstring, "hello"); assert_eq!(test_ref_3.get().unwrap().vu32, None); assert_eq!(test_ref_3.get().unwrap().vbool, None); @@ -272,14 +292,13 @@ pub(crate) mod tests { let manager = StoreManager::new(FsOptions::Local, vec![]).unwrap(); let base_fs = manager.base_fs(); - let record_batch = get_test_record_batch::( - DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), - TokioExecutor::new(), - ) - .await; + let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); + let record_batch = + get_test_record_batch::(option.clone(), TokioExecutor::new()).await; let table_path = temp_dir.path().join("projection_scan_test.parquet"); let _ = File::create(&table_path).unwrap(); let table_path = Path::from_filesystem_path(table_path).unwrap(); + let table_gen = FileId::new(); let file = base_fs .open_options(&table_path, FileType::Parquet.open_options(false)) @@ -288,19 +307,20 @@ pub(crate) mod tests { write_record_batch(file, &record_batch).await.unwrap(); { - let mut test_ref_1 = open_sstable::(base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 3], - ), - ) - .await - .unwrap(); + let mut test_ref_1 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 3], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_1.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -313,19 +333,20 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_2 = open_sstable::(base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2, 4], - ), - ) - .await - .unwrap(); + let mut test_ref_2 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2, 4], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_2.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); @@ -338,19 +359,20 @@ pub(crate) mod tests { assert_eq!(entry_1.get().unwrap().vbool, None); } { - let mut test_ref_3 = open_sstable::(base_fs, table_path.clone()) - .await - .scan( - (Bound::Unbounded, Bound::Unbounded), - 1_u32.into(), - None, - ProjectionMask::roots( - &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), - [0, 1, 2], - ), - ) - .await - .unwrap(); + let mut test_ref_3 = + open_sstable::(base_fs, table_path.clone(), table_gen, &option) + .await + .scan( + (Bound::Unbounded, Bound::Unbounded), + 1_u32.into(), + None, + ProjectionMask::roots( + &arrow_to_parquet_schema(Test::arrow_schema()).unwrap(), + [0, 1, 2], + ), + ) + .await + .unwrap(); let entry_0 = test_ref_3.next().await.unwrap().unwrap(); assert_eq!(entry_0.get().unwrap().vstring, "hello"); diff --git a/src/option.rs b/src/option.rs index 4efbcb85..3f90a57d 100644 --- a/src/option.rs +++ b/src/option.rs @@ -11,6 +11,7 @@ use parquet::{ format::SortingColumn, schema::types::ColumnPath, }; +use tonbo_ext_reader::CacheReader; use crate::{ fs::{FileId, FileType}, @@ -281,8 +282,12 @@ where self.level_paths[level].as_ref().map(|(path, _)| path) } - pub(crate) fn is_threshold_exceeded_major(&self, version: &Version, level: usize) -> bool { - Version::::tables_len(version, level) + pub(crate) fn is_threshold_exceeded_major( + &self, + version: &Version, + level: usize, + ) -> bool { + Version::::tables_len(version, level) >= (self.major_threshold_with_sst_size * self.level_sst_magnification.pow(level as u32)) } } diff --git a/src/stream/level.rs b/src/stream/level.rs index ebcdb688..b41326ec 100644 --- a/src/stream/level.rs +++ b/src/stream/level.rs @@ -13,7 +13,8 @@ use fusio::{ }; use futures_core::Stream; use parquet::{arrow::ProjectionMask, errors::ParquetError}; -use tonbo_ext_reader::{MetaCache, RangeCache}; +use pin_project_lite::pin_project; +use tonbo_ext_reader::CacheReader; use crate::{ fs::{FileId, FileType}, @@ -26,46 +27,57 @@ use crate::{ DbOption, }; -enum FutureStatus<'level, R> +enum FutureStatus<'level, R, C> where R: Record, + C: CacheReader, { Init(FileId), - Ready(SsTableScan<'level, R>), + Ready(SsTableScan<'level, R, C>), OpenFile(Pin, Error>> + 'level>>), - OpenSst(Pin, Error>> + Send + 'level>>), + OpenSst(Pin, Error>> + Send + 'level>>), LoadStream( - Pin, ParquetError>> + Send + 'level>>, + Pin< + Box< + dyn Future, ParquetError>> + + Send + + 'level, + >, + >, ), } -pub(crate) struct LevelStream<'level, R> -where - R: Record, -{ - lower: Bound<&'level R::Key>, - upper: Bound<&'level R::Key>, - ts: Timestamp, - level: usize, - option: Arc>, - meta_cache: MetaCache, - range_cache: RangeCache, - gens: VecDeque, - limit: Option, - projection_mask: ProjectionMask, - status: FutureStatus<'level, R>, - fs: Arc, - path: Option<(Path, FileId)>, +pin_project! { + pub(crate) struct LevelStream<'level, R, C> + where + R: Record, + C: CacheReader, + { + lower: Bound<&'level R::Key>, + upper: Bound<&'level R::Key>, + ts: Timestamp, + level: usize, + option: Arc>, + meta_cache: C::MetaCache, + range_cache: C::RangeCache, + gens: VecDeque, + limit: Option, + projection_mask: ProjectionMask, + status: FutureStatus<'level, R, C>, + fs: Arc, + path: Option<(Path, FileId)>, + } } -impl<'level, R> LevelStream<'level, R> +impl<'level, R, C> LevelStream<'level, R, C> where R: Record, + C: CacheReader, { // Kould: only used by Compaction now, and the start and end of the sstables range are known #[allow(clippy::too_many_arguments)] pub(crate) fn new( - version: &Version, + version: &Version, level: usize, start: usize, end: usize, @@ -101,21 +113,24 @@ where } } -impl<'level, R> Stream for LevelStream<'level, R> +impl<'level, R, C> Stream for LevelStream<'level, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result, ParquetError>; - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { + let mut this = self.project(); + loop { - return match &mut self.status { + return match &mut this.status { FutureStatus::Init(gen) => { let gen = *gen; - self.path = Some((self.option.table_path(&gen, self.level), gen)); + *this.path = Some((this.option.table_path(&gen, *this.level), gen)); - let reader = self.fs.open_options( - &self.path.as_ref().unwrap().0, + let reader = this.fs.open_options( + &this.path.as_ref().unwrap().0, FileType::Parquet.open_options(true), ); #[allow(clippy::missing_transmute_annotations)] @@ -130,17 +145,17 @@ where >, >(reader) }; - self.status = FutureStatus::OpenFile(reader); + *this.status = FutureStatus::OpenFile(reader); continue; } FutureStatus::Ready(stream) => match Pin::new(stream).poll_next(cx) { - Poll::Ready(None) => match self.gens.pop_front() { + Poll::Ready(None) => match this.gens.pop_front() { None => Poll::Ready(None), Some(gen) => { - self.path = Some((self.option.table_path(&gen, self.level), gen)); + *this.path = Some((this.option.table_path(&gen, *this.level), gen)); - let reader = self.fs.open_options( - &self.path.as_ref().unwrap().0, + let reader = this.fs.open_options( + &this.path.as_ref().unwrap().0, FileType::Parquet.open_options(true), ); #[allow(clippy::missing_transmute_annotations)] @@ -156,12 +171,12 @@ where >, >(reader) }; - self.status = FutureStatus::OpenFile(reader); + *this.status = FutureStatus::OpenFile(reader); continue; } }, Poll::Ready(Some(result)) => { - if let Some(limit) = &mut self.limit { + if let Some(limit) = &mut this.limit { *limit -= 1; } Poll::Ready(Some(result)) @@ -170,12 +185,12 @@ where }, FutureStatus::OpenFile(file_future) => match Pin::new(file_future).poll(cx) { Poll::Ready(Ok(file)) => { - let meta_cache = self.meta_cache.clone(); - let range_cache = self.range_cache.clone(); - let (_, gen) = self.path.clone().unwrap(); + let meta_cache = this.meta_cache.clone(); + let range_cache = this.range_cache.clone(); + let (_, gen) = this.path.clone().unwrap(); let future = async move { SsTable::open(file, gen, range_cache, meta_cache).await }; - self.status = FutureStatus::OpenSst(Box::pin(future)); + *this.status = FutureStatus::OpenSst(Box::pin(future)); continue; } Poll::Ready(Err(err)) => { @@ -185,11 +200,11 @@ where }, FutureStatus::OpenSst(sst_future) => match Pin::new(sst_future).poll(cx) { Poll::Ready(Ok(sst)) => { - self.status = FutureStatus::LoadStream(Box::pin(sst.scan( - (self.lower, self.upper), - self.ts, - self.limit, - self.projection_mask.clone(), + *this.status = FutureStatus::LoadStream(Box::pin(sst.scan( + (*this.lower, *this.upper), + *this.ts, + *this.limit, + this.projection_mask.clone(), ))); continue; } @@ -200,7 +215,7 @@ where }, FutureStatus::LoadStream(stream_future) => match Pin::new(stream_future).poll(cx) { Poll::Ready(Ok(scan)) => { - self.status = FutureStatus::Ready(scan); + *this.status = FutureStatus::Ready(scan); continue; } Poll::Ready(Err(err)) => Poll::Ready(Some(Err(err))), diff --git a/src/stream/mem_projection.rs b/src/stream/mem_projection.rs index 6671e285..92062940 100644 --- a/src/stream/mem_projection.rs +++ b/src/stream/mem_projection.rs @@ -7,6 +7,7 @@ use std::{ use futures_core::Stream; use parquet::{arrow::ProjectionMask, errors::ParquetError}; use pin_project_lite::pin_project; +use tonbo_ext_reader::CacheReader; use crate::{ record::Record, @@ -14,20 +15,25 @@ use crate::{ }; pin_project! { - pub struct MemProjectionStream<'projection, R> + pub struct MemProjectionStream<'projection, R, C> where R: Record, + C: CacheReader { - stream: Box>, + stream: Box>, projection_mask: Arc, } } -impl<'projection, R> MemProjectionStream<'projection, R> +impl<'projection, R, C> MemProjectionStream<'projection, R, C> where R: Record, + C: CacheReader, { - pub(crate) fn new(stream: ScanStream<'projection, R>, projection_mask: ProjectionMask) -> Self { + pub(crate) fn new( + stream: ScanStream<'projection, R, C>, + projection_mask: ProjectionMask, + ) -> Self { Self { stream: Box::new(stream), projection_mask: Arc::new(projection_mask), @@ -35,9 +41,10 @@ where } } -impl<'projection, R> Stream for MemProjectionStream<'projection, R> +impl<'projection, R, C> Stream for MemProjectionStream<'projection, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result, ParquetError>; @@ -61,6 +68,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; use parquet::arrow::{arrow_to_parquet_schema, ProjectionMask}; + use tonbo_ext_reader::foyer_reader::FoyerReader; use crate::{ inmem::mutable::Mutable, record::Record, stream::mem_projection::MemProjectionStream, @@ -121,7 +129,7 @@ mod tests { vec![0, 1, 2, 4], ); - let mut stream = MemProjectionStream::::new( + let mut stream = MemProjectionStream::::new( mutable .scan((Bound::Unbounded, Bound::Unbounded), 6.into()) .into(), diff --git a/src/stream/merge.rs b/src/stream/merge.rs index f58583c6..7d144a6f 100644 --- a/src/stream/merge.rs +++ b/src/stream/merge.rs @@ -8,16 +8,18 @@ use std::{ use futures_core::{ready, Stream}; use futures_util::stream::StreamExt; use pin_project_lite::pin_project; +use tonbo_ext_reader::CacheReader; use super::{Entry, ScanStream}; use crate::{record::Record, timestamp::Timestamp}; pin_project! { - pub struct MergeStream<'merge, R> + pub struct MergeStream<'merge, R, C> where R: Record, + C: CacheReader, { - streams: Vec>, + streams: Vec>, peeked: BinaryHeap>, buf: Option>, ts: Timestamp, @@ -25,12 +27,13 @@ pin_project! { } } -impl<'merge, R> MergeStream<'merge, R> +impl<'merge, R, C> MergeStream<'merge, R, C> where R: Record, + C: CacheReader + 'static, { pub(crate) async fn from_vec( - mut streams: Vec>, + mut streams: Vec>, ts: Timestamp, ) -> Result { let mut peeked = BinaryHeap::with_capacity(streams.len()); @@ -62,9 +65,10 @@ where } } -impl<'merge, R> Stream for MergeStream<'merge, R> +impl<'merge, R, C> Stream for MergeStream<'merge, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result, parquet::errors::ParquetError>; @@ -160,6 +164,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; + use tonbo_ext_reader::foyer_reader::FoyerReader; use super::MergeStream; use crate::{ @@ -212,7 +217,7 @@ mod tests { let lower = "a".to_string(); let upper = "e".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![ m1.scan(bound, 6.into()).into(), m2.scan(bound, 6.into()).into(), @@ -291,10 +296,12 @@ mod tests { let lower = "1".to_string(); let upper = "4".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = - MergeStream::::from_vec(vec![m1.scan(bound, 0.into()).into()], 0.into()) - .await - .unwrap(); + let mut merge = MergeStream::::from_vec( + vec![m1.scan(bound, 0.into()).into()], + 0.into(), + ) + .await + .unwrap(); if let Some(Ok(Entry::Mutable(entry))) = merge.next().await { assert_eq!(entry.key().value, "1"); @@ -319,10 +326,12 @@ mod tests { let lower = "1".to_string(); let upper = "4".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = - MergeStream::::from_vec(vec![m1.scan(bound, 1.into()).into()], 1.into()) - .await - .unwrap(); + let mut merge = MergeStream::::from_vec( + vec![m1.scan(bound, 1.into()).into()], + 1.into(), + ) + .await + .unwrap(); if let Some(Ok(Entry::Mutable(entry))) = merge.next().await { assert_eq!(entry.key().value, "1"); @@ -371,7 +380,7 @@ mod tests { let lower = "1".to_string(); let upper = "3".to_string(); { - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Included(&lower), Bound::Included(&upper)), 0.into()) .into()], @@ -391,7 +400,7 @@ mod tests { assert!(merge.next().await.is_none()); } { - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Included(&lower), Bound::Included(&upper)), 0.into()) .into()], diff --git a/src/stream/mod.rs b/src/stream/mod.rs index fa0b5afe..c42db2ae 100644 --- a/src/stream/mod.rs +++ b/src/stream/mod.rs @@ -17,6 +17,7 @@ use futures_util::{ready, stream}; use parquet::arrow::ProjectionMask; use pin_project_lite::pin_project; use record_batch::RecordBatchEntry; +use tonbo_ext_reader::CacheReader; use crate::{ inmem::{immutable::ImmutableScan, mutable::MutableScan}, @@ -100,9 +101,10 @@ where pin_project! { #[project = ScanStreamProject] - pub enum ScanStream<'scan, R> + pub enum ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { Transaction { #[pin] @@ -118,22 +120,23 @@ pin_project! { }, SsTable { #[pin] - inner: SsTableScan<'scan, R>, + inner: SsTableScan<'scan, R, C>, }, Level { #[pin] - inner: LevelStream<'scan, R>, + inner: LevelStream<'scan, R, C>, }, MemProjection { #[pin] - inner: MemProjectionStream<'scan, R>, + inner: MemProjectionStream<'scan, R, C>, } } } -impl<'scan, R> From> for ScanStream<'scan, R> +impl<'scan, R, C> From> for ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { fn from(inner: TransactionScan<'scan, R>) -> Self { ScanStream::Transaction { @@ -142,9 +145,10 @@ where } } -impl<'scan, R> From> for ScanStream<'scan, R> +impl<'scan, R, C> From> for ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { fn from(inner: MutableScan<'scan, R>) -> Self { ScanStream::Mutable { @@ -153,9 +157,10 @@ where } } -impl<'scan, R> From> for ScanStream<'scan, R> +impl<'scan, R, C> From> for ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { fn from(inner: ImmutableScan<'scan, R>) -> Self { ScanStream::Immutable { @@ -164,27 +169,30 @@ where } } -impl<'scan, R> From> for ScanStream<'scan, R> +impl<'scan, R, C> From> for ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { - fn from(inner: SsTableScan<'scan, R>) -> Self { + fn from(inner: SsTableScan<'scan, R, C>) -> Self { ScanStream::SsTable { inner } } } -impl<'scan, R> From> for ScanStream<'scan, R> +impl<'scan, R, C> From> for ScanStream<'scan, R, C> where R: Record, + C: CacheReader, { - fn from(inner: MemProjectionStream<'scan, R>) -> Self { + fn from(inner: MemProjectionStream<'scan, R, C>) -> Self { ScanStream::MemProjection { inner } } } -impl fmt::Debug for ScanStream<'_, R> +impl fmt::Debug for ScanStream<'_, R, C> where R: Record, + C: CacheReader, { fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { match self { @@ -198,9 +206,10 @@ where } } -impl<'scan, R> Stream for ScanStream<'scan, R> +impl<'scan, R, C> Stream for ScanStream<'scan, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result, parquet::errors::ParquetError>; diff --git a/src/stream/package.rs b/src/stream/package.rs index c8493e8f..e1a4f20d 100644 --- a/src/stream/package.rs +++ b/src/stream/package.rs @@ -5,6 +5,7 @@ use std::{ use futures_core::Stream; use pin_project_lite::pin_project; +use tonbo_ext_reader::CacheReader; use crate::{ inmem::immutable::{ArrowArrays, Builder}, @@ -13,25 +14,27 @@ use crate::{ }; pin_project! { - pub struct PackageStream<'package, R> + pub struct PackageStream<'package, R, C> where R: Record, + C: CacheReader, { row_count: usize, batch_size: usize, - inner: MergeStream<'package, R>, + inner: MergeStream<'package, R, C>, builder: ::Builder, projection_indices: Option>, } } -impl<'package, R> PackageStream<'package, R> +impl<'package, R, C> PackageStream<'package, R, C> where R: Record, + C: CacheReader, { pub(crate) fn new( batch_size: usize, - merge: MergeStream<'package, R>, + merge: MergeStream<'package, R, C>, projection_indices: Option>, instance: &RecordInstance, ) -> Self { @@ -45,9 +48,10 @@ where } } -impl<'package, R> Stream for PackageStream<'package, R> +impl<'package, R, C> Stream for PackageStream<'package, R, C> where R: Record, + C: CacheReader + 'static, { type Item = Result; @@ -85,6 +89,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; use tempfile::TempDir; + use tonbo_ext_reader::foyer_reader::FoyerReader; use crate::{ inmem::{ @@ -177,7 +182,7 @@ mod tests { .await .unwrap(); - let merge = MergeStream::::from_vec( + let merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Unbounded, Bound::Unbounded), 6.into()) .into()], diff --git a/src/transaction.rs b/src/transaction.rs index 3a9d38c1..8e85b340 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -13,6 +13,7 @@ use flume::SendError; use lockable::AsyncLimit; use parquet::{arrow::ProjectionMask, errors::ParquetError}; use thiserror::Error; +use tonbo_ext_reader::CacheReader; use crate::{ compaction::CompactTask, @@ -45,24 +46,26 @@ where } /// optimistic ACID transaction, open with /// [`DB::transaction`](crate::DB::transaction) method -pub struct Transaction<'txn, R> +pub struct Transaction<'txn, R, C> where R: Record, + C: CacheReader, { ts: Timestamp, local: BTreeMap>, share: RwLockReadGuard<'txn, Schema>, - version: VersionRef, + version: VersionRef, lock_map: LockMap, manager: Arc, } -impl<'txn, R> Transaction<'txn, R> +impl<'txn, R, C> Transaction<'txn, R, C> where R: Record + Send, + C: CacheReader + 'static, { pub(crate) fn new( - version: VersionRef, + version: VersionRef, share: RwLockReadGuard<'txn, Schema>, lock_map: LockMap, manager: Arc, @@ -104,7 +107,7 @@ where pub fn scan<'scan>( &'scan self, range: (Bound<&'scan R::Key>, Bound<&'scan R::Key>), - ) -> Scan<'scan, R> { + ) -> Scan<'scan, R, C> { Scan::new( &self.share, &self.manager, @@ -257,6 +260,7 @@ mod tests { use fusio_dispatch::FsOptions; use futures_util::StreamExt; use tempfile::TempDir; + use tonbo_ext_reader::foyer_reader::FoyerReader; use crate::{ compaction::tests::build_version, @@ -276,7 +280,7 @@ mod tests { async fn transaction_read_write() { let temp_dir = TempDir::new().unwrap(); - let db = DB::::new( + let db = DB::::new( DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), ) @@ -404,7 +408,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db = DB::::new(option, TokioExecutor::new()) + let db = DB::::new(option, TokioExecutor::new()) .await .unwrap(); @@ -437,7 +441,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db = DB::::new(option, TokioExecutor::new()) + let db = DB::::new(option, TokioExecutor::new()) .await .unwrap(); @@ -807,7 +811,7 @@ mod tests { "age".to_string(), 0, ); - let db = DB::with_schema(option, TokioExecutor::default(), descs, 0) + let db = DB::<_, _, FoyerReader>::with_schema(option, TokioExecutor::default(), descs, 0) .await .unwrap(); diff --git a/src/version/mod.rs b/src/version/mod.rs index b7bd9c68..7ee5eeab 100644 --- a/src/version/mod.rs +++ b/src/version/mod.rs @@ -3,6 +3,7 @@ pub(crate) mod edit; pub(crate) mod set; use std::{ + fmt::{Debug, Formatter}, ops::Bound, sync::{ atomic::{AtomicU32, Ordering}, @@ -14,7 +15,7 @@ use flume::{SendError, Sender}; use fusio::DynFs; use parquet::arrow::ProjectionMask; use thiserror::Error; -use tonbo_ext_reader::{CacheError, MetaCache, RangeCache}; +use tonbo_ext_reader::{CacheError, CacheReader}; use tracing::error; use crate::{ @@ -31,7 +32,7 @@ use crate::{ pub(crate) const MAX_LEVEL: usize = 7; -pub(crate) type VersionRef = Arc>; +pub(crate) type VersionRef = Arc>; pub(crate) trait TransactionTs { fn load_ts(&self) -> Timestamp; @@ -39,10 +40,10 @@ pub(crate) trait TransactionTs { fn increase_ts(&self) -> Timestamp; } -#[derive(Debug)] -pub(crate) struct Version +pub(crate) struct Version where R: Record, + C: CacheReader, { ts: Timestamp, pub(crate) level_slice: [Vec>; MAX_LEVEL], @@ -51,21 +52,41 @@ where timestamp: Arc, log_length: u32, - range_cache: RangeCache, - meta_cache: MetaCache, + range_cache: C::RangeCache, + meta_cache: C::MetaCache, } -impl Version +impl Debug for Version where R: Record, + C: CacheReader, +{ + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("Version") + .field("ts", &self.ts) + .field("level_slice", &self.level_slice) + .field("clean_sender", &format_args!("{:?}", self.clean_sender)) + .field("option", &format_args!("{:?}", self.option)) + .field("timestamp", &format_args!("{:?}", self.timestamp)) + .field("log_length", &format_args!("{}", self.log_length)) + .field("range_cache", &format_args!("{:?}", self.range_cache)) + .field("meta_cache", &format_args!("{:?}", self.meta_cache)) + .finish() + } +} + +impl Version +where + R: Record, + C: CacheReader, { #[cfg(test)] pub(crate) fn new( option: Arc>, clean_sender: Sender, timestamp: Arc, - range_cache: RangeCache, - meta_cache: MetaCache, + range_cache: C::RangeCache, + meta_cache: C::MetaCache, ) -> Self { Version { ts: Timestamp::from(0), @@ -83,17 +104,18 @@ where &self.option } - pub(crate) fn meta_cache(&self) -> MetaCache { + pub(crate) fn meta_cache(&self) -> C::MetaCache { self.meta_cache.clone() } - pub(crate) fn range_cache(&self) -> RangeCache { + pub(crate) fn range_cache(&self) -> C::RangeCache { self.range_cache.clone() } } -impl TransactionTs for Version +impl TransactionTs for Version where R: Record, + C: CacheReader, { fn load_ts(&self) -> Timestamp { self.timestamp.load(Ordering::Acquire).into() @@ -104,9 +126,10 @@ where } } -impl Clone for Version +impl Clone for Version where R: Record, + C: CacheReader, { fn clone(&self) -> Self { let mut level_slice = [const { Vec::new() }; MAX_LEVEL]; @@ -128,9 +151,10 @@ where } } -impl Version +impl Version where R: Record, + C: CacheReader + 'static, { pub(crate) async fn query( &self, @@ -198,7 +222,7 @@ where .open_options(&path, FileType::Parquet.open_options(true)) .await .map_err(VersionError::Fusio)?; - SsTable::::open(file, *gen, self.range_cache(), self.meta_cache()) + SsTable::::open(file, *gen, self.range_cache(), self.meta_cache()) .await? .get(key, projection_mask) .await @@ -218,7 +242,7 @@ where pub(crate) async fn streams<'streams>( &self, manager: &StoreManager, - streams: &mut Vec>, + streams: &mut Vec>, range: (Bound<&'streams R::Key>, Bound<&'streams R::Key>), ts: Timestamp, limit: Option, @@ -308,9 +332,10 @@ where } } -impl Drop for Version +impl Drop for Version where R: Record, + C: CacheReader, { fn drop(&mut self) { if let Err(err) = self.clean_sender.send(CleanTag::Clean { ts: self.ts }) { diff --git a/src/version/set.rs b/src/version/set.rs index 8793b264..8a500b7f 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -12,7 +12,7 @@ use async_lock::RwLock; use flume::Sender; use fusio::{dynamic::DynFile, fs::FileMeta, path::path_to_local}; use futures_util::StreamExt; -use tonbo_ext_reader::{foyer_reader::build_cache, MetaCache, RangeCache}; +use tonbo_ext_reader::CacheReader; use super::{TransactionTs, MAX_LEVEL}; use crate::{ @@ -46,31 +46,34 @@ impl Ord for CmpMeta { } } -pub(crate) struct VersionSetInner +pub(crate) struct VersionSetInner where R: Record, + C: CacheReader, { - current: VersionRef, + current: VersionRef, log_with_id: (Box, FileId), } -pub(crate) struct VersionSet +pub(crate) struct VersionSet where R: Record, + C: CacheReader, { - inner: Arc>>, + inner: Arc>>, clean_sender: Sender, timestamp: Arc, option: Arc>, manager: Arc, - range_cache: RangeCache, - meta_cache: MetaCache, + range_cache: C::RangeCache, + meta_cache: C::MetaCache, } -impl Clone for VersionSet +impl Clone for VersionSet where R: Record, + C: CacheReader, { fn clone(&self) -> Self { VersionSet { @@ -85,9 +88,10 @@ where } } -impl TransactionTs for VersionSet +impl TransactionTs for VersionSet where R: Record, + C: CacheReader, { fn load_ts(&self) -> Timestamp { self.timestamp.load(Ordering::Acquire).into() @@ -98,9 +102,10 @@ where } } -impl VersionSet +impl VersionSet where R: Record, + C: CacheReader + 'static, { pub(crate) async fn new( clean_sender: Sender, @@ -155,7 +160,7 @@ where let timestamp = Arc::new(AtomicU32::default()); drop(log_stream); - let (meta_cache, range_cache) = build_cache( + let (meta_cache, range_cache) = C::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -165,9 +170,9 @@ where ) .await?; - let set = VersionSet:: { + let set = VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { - current: Arc::new(Version:: { + current: Arc::new(Version:: { ts: Timestamp::from(0), level_slice: [const { Vec::new() }; MAX_LEVEL], clean_sender: clean_sender.clone(), @@ -191,7 +196,7 @@ where Ok(set) } - pub(crate) async fn current(&self) -> VersionRef { + pub(crate) async fn current(&self) -> VersionRef { self.inner.read().await.current.clone() } @@ -315,7 +320,7 @@ pub(crate) mod tests { use fusio_dispatch::FsOptions; use futures_util::StreamExt; use tempfile::TempDir; - use tonbo_ext_reader::foyer_reader::build_cache; + use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; use crate::{ fs::{manager::StoreManager, FileId, FileType}, @@ -330,14 +335,15 @@ pub(crate) mod tests { DbOption, }; - pub(crate) async fn build_version_set( - version: Version, + pub(crate) async fn build_version_set( + version: Version, clean_sender: Sender, option: Arc>, manager: Arc, - ) -> Result, VersionError> + ) -> Result, VersionError> where R: Record, + C: CacheReader, { let log_id = FileId::new(); let log = manager @@ -348,7 +354,7 @@ pub(crate) mod tests { ) .await?; let timestamp = version.timestamp.clone(); - let (meta_cache, range_cache) = build_cache( + let (meta_cache, range_cache) = C::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -358,7 +364,7 @@ pub(crate) mod tests { ) .await?; - Ok(VersionSet:: { + Ok(VersionSet:: { inner: Arc::new(RwLock::new(VersionSetInner { current: Arc::new(version), log_with_id: (log, log_id), @@ -386,7 +392,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager.clone()) .await .unwrap(); @@ -402,7 +408,7 @@ pub(crate) mod tests { drop(version_set); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager) .await .unwrap(); @@ -424,7 +430,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager.clone()) .await .unwrap(); @@ -552,7 +558,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager) .await .unwrap(); diff --git a/tests/data_integrity.rs b/tests/data_integrity.rs index 239d0d83..0cd2b39f 100644 --- a/tests/data_integrity.rs +++ b/tests/data_integrity.rs @@ -6,6 +6,7 @@ mod tests { use futures_util::StreamExt; use tempfile::TempDir; use tonbo::{executor::tokio::TokioExecutor, DbOption, Record, DB}; + use tonbo_ext_reader::foyer_reader::FoyerReader; const WRITE_TIMES: usize = 500_000; const STRING_SIZE: usize = 50; @@ -72,7 +73,8 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); + let db: DB = + DB::new(option, TokioExecutor::new()).await.unwrap(); for _ in 0..WRITE_TIMES { let customer = gen_record(&mut rng, &mut primary_key_count); diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml index d75f344a..1e792d35 100644 --- a/tonbo_ext_reader/Cargo.toml +++ b/tonbo_ext_reader/Cargo.toml @@ -11,6 +11,7 @@ bytes = { version = "1.7", features = ["serde"] } foyer = { version = "0.12" } futures-core = "0.3" futures-util = "0.3" +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } parquet = { version = "53", features = ["async"] } thiserror = "1" ulid = { version = "1", features = ["serde"] } @@ -27,6 +28,5 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-dispatch", version = "0.2.0", features = [ "tokio", ] } -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } tempfile = "3" tokio = { version = "1", features = ["full"] } \ No newline at end of file diff --git a/tonbo_ext_reader/src/foyer_reader.rs b/tonbo_ext_reader/src/foyer_reader.rs index c8b76de2..4e79f544 100644 --- a/tonbo_ext_reader/src/foyer_reader.rs +++ b/tonbo_ext_reader/src/foyer_reader.rs @@ -1,28 +1,59 @@ use std::{ops::Range, sync::Arc}; use bytes::Bytes; -use foyer::{CacheBuilder, DirectFsDeviceOptions, Engine, HybridCacheBuilder, LruConfig}; +use foyer::{ + Cache, CacheBuilder, DirectFsDeviceOptions, Engine, HybridCache, HybridCacheBuilder, LruConfig, +}; +use fusio_parquet::reader::AsyncReader; use futures_core::future::BoxFuture; use futures_util::FutureExt; use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; use ulid::Ulid; -use crate::{CacheError, MetaCache, RangeCache}; +use crate::{CacheError, CacheReader, MetaCache, RangeCache}; -pub struct CacheReader { +#[derive(Debug, Clone)] +pub struct FoyerMetaCache(Cache>); +#[derive(Debug, Clone)] +pub struct FoyerRangeCache(HybridCache<(Ulid, Range), Bytes>); + +pub struct FoyerReader { gen: Ulid, - inner: R, - range_cache: RangeCache, - meta_cache: MetaCache, + inner: AsyncReader, + range_cache: FoyerRangeCache, + meta_cache: FoyerMetaCache, } -impl CacheReader { - pub fn new( - meta_cache: MetaCache, - range_cache: RangeCache, +impl MetaCache for FoyerMetaCache { + fn get(&self, gen: &Ulid) -> Option> { + self.0.get(gen).map(|entry| entry.value().clone()) + } + + fn insert(&self, gen: Ulid, data: Arc) -> Arc { + self.0.insert(gen, data).value().clone() + } +} + +impl RangeCache for FoyerRangeCache { + async fn get(&self, key: &(Ulid, Range)) -> Result, CacheError> { + Ok(self.0.get(key).await?.map(|entry| entry.value().clone())) + } + + fn insert(&self, key: (Ulid, Range), bytes: Bytes) -> Bytes { + self.0.insert(key, bytes).value().clone() + } +} + +impl CacheReader for FoyerReader { + type MetaCache = FoyerMetaCache; + type RangeCache = FoyerRangeCache; + + fn new( + meta_cache: Self::MetaCache, + range_cache: Self::RangeCache, gen: Ulid, - inner: R, - ) -> CacheReader { + inner: AsyncReader, + ) -> Self { Self { gen, inner, @@ -30,58 +61,57 @@ impl CacheReader { meta_cache, } } -} -pub async fn build_cache( - cache_path: impl AsRef, - cache_meta_capacity: usize, - cache_meta_shards: usize, - cache_meta_ratio: f64, - cache_range_memory: usize, - cache_range_disk: usize, -) -> Result<(MetaCache, RangeCache), CacheError> { - let meta_cache = Arc::new( - CacheBuilder::new(cache_meta_capacity) + async fn build_caches( + cache_path: impl AsRef + Send, + cache_meta_capacity: usize, + cache_meta_shards: usize, + cache_meta_ratio: f64, + cache_range_memory: usize, + cache_range_disk: usize, + ) -> Result<(Self::MetaCache, Self::RangeCache), CacheError> { + let meta_cache = CacheBuilder::new(cache_meta_capacity) .with_shards(cache_meta_shards) .with_eviction_config(LruConfig { high_priority_pool_ratio: cache_meta_ratio, }) - .build(), - ); - let range_cache = HybridCacheBuilder::new() - .memory(cache_range_memory) - .storage(Engine::Large) - .with_device_options(DirectFsDeviceOptions::new(cache_path).with_capacity(cache_range_disk)) - .build() - .await - .map_err(CacheError::from)?; - Ok((meta_cache, range_cache)) + .build(); + let range_cache = HybridCacheBuilder::new() + .memory(cache_range_memory) + .storage(Engine::Large) + .with_device_options( + DirectFsDeviceOptions::new(cache_path).with_capacity(cache_range_disk), + ) + .build() + .await + .map_err(CacheError::from)?; + Ok((FoyerMetaCache(meta_cache), FoyerRangeCache(range_cache))) + } } -impl AsyncFileReader for CacheReader { +impl AsyncFileReader for FoyerReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { async move { let key = (self.gen, range); - if let Some(entry) = self + if let Some(bytes) = self .range_cache .get(&key) .await .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? { - return Ok(entry.value().clone()); + return Ok(bytes); } let bytes = self.inner.get_bytes(key.1.clone()).await?; - let entry = self.range_cache.insert(key, bytes); - Ok(entry.value().clone()) + Ok(self.range_cache.insert(key, bytes)) } .boxed() } fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { async move { - if let Some(entry) = self.meta_cache.get(&self.gen) { - return Ok(entry.value().clone()); + if let Some(meta) = self.meta_cache.get(&self.gen) { + return Ok(meta); } let meta = self @@ -89,9 +119,8 @@ impl AsyncFileReader for CacheReader { .get_metadata() .await .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; - let entry = self.meta_cache.insert(self.gen, meta); - Ok(entry.value().clone()) + Ok(self.meta_cache.insert(self.gen, meta)) } .boxed() } @@ -121,7 +150,7 @@ pub(crate) mod tests { use tempfile::TempDir; use ulid::Ulid; - use crate::foyer_reader::{build_cache, CacheReader}; + use crate::{foyer_reader::FoyerReader, CacheReader}; struct CountFile { inner: Box, @@ -208,7 +237,7 @@ pub(crate) mod tests { writer.close().await.unwrap(); let read_count = Arc::new(AtomicUsize::new(0)); - let (meta_cache, range_cache) = build_cache( + let (meta_cache, range_cache) = FoyerReader::build_caches( temp_dir.path().join("cache"), 32, 4, @@ -227,7 +256,7 @@ pub(crate) mod tests { .unwrap(); let content_len = file.size().await.unwrap(); - let mut reader = CacheReader::new( + let mut reader = FoyerReader::new( meta_cache.clone(), range_cache.clone(), gen, diff --git a/tonbo_ext_reader/src/lib.rs b/tonbo_ext_reader/src/lib.rs index 5c64128d..e9cbab10 100644 --- a/tonbo_ext_reader/src/lib.rs +++ b/tonbo_ext_reader/src/lib.rs @@ -1,16 +1,48 @@ -use std::{io, ops::Range, sync::Arc}; +use std::{fmt::Debug, io, ops::Range, sync::Arc}; use bytes::Bytes; -use foyer::{Cache, HybridCache}; -use parquet::file::metadata::ParquetMetaData; +use fusio_parquet::reader::AsyncReader; +use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; use thiserror::Error; use ulid::Ulid; pub mod foyer_reader; -// Tips: Conditional compilation adapts to different implementations -pub type MetaCache = Arc>>; -pub type RangeCache = HybridCache<(Ulid, Range), Bytes>; +pub trait MetaCache: Sync + Send + Clone + Debug { + fn get(&self, gen: &Ulid) -> Option>; + + fn insert(&self, gen: Ulid, data: Arc) -> Arc; +} + +pub trait RangeCache: Sync + Send + Clone + Debug { + fn get( + &self, + key: &(Ulid, Range), + ) -> impl std::future::Future, CacheError>> + Send; + + fn insert(&self, key: (Ulid, Range), bytes: Bytes) -> Bytes; +} + +pub trait CacheReader: AsyncFileReader + Unpin { + type MetaCache: MetaCache; + type RangeCache: RangeCache; + + fn new( + meta_cache: Self::MetaCache, + range_cache: Self::RangeCache, + gen: Ulid, + inner: AsyncReader, + ) -> Self; + + fn build_caches( + cache_path: impl AsRef + Send, + cache_meta_capacity: usize, + cache_meta_shards: usize, + cache_meta_ratio: f64, + cache_range_memory: usize, + cache_range_disk: usize, + ) -> impl std::future::Future> + Send; +} #[derive(Debug, Error)] pub enum CacheError { From 06af52bfe236361ae18514bacb810e0ed7bb649e Mon Sep 17 00:00:00 2001 From: Kould Date: Thu, 31 Oct 2024 17:48:10 +0800 Subject: [PATCH 08/16] chore: fix benchmark --- benches/common.rs | 13 +++++++------ benches/criterion/writes.rs | 3 ++- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/benches/common.rs b/benches/common.rs index 3abb7c5b..537da116 100644 --- a/benches/common.rs +++ b/benches/common.rs @@ -16,6 +16,7 @@ use tokio::fs::create_dir_all; use tonbo::{ executor::tokio::TokioExecutor, stream, transaction::TransactionEntry, DbOption, Projection, }; +use tonbo_ext_reader::foyer_reader::FoyerReader; use tonbo_macros::Record; const RNG_SEED: u64 = 3; @@ -185,12 +186,12 @@ pub trait BenchReader { } pub struct TonboBenchDataBase { - db: tonbo::DB, + db: tonbo::DB, } impl TonboBenchDataBase { #[allow(dead_code)] - pub fn new(db: tonbo::DB) -> Self { + pub fn new(db: tonbo::DB) -> Self { TonboBenchDataBase { db } } } @@ -234,7 +235,7 @@ impl BenchDatabase for TonboBenchDataBase { } pub struct TonboBenchReadTransaction<'a> { - txn: tonbo::transaction::Transaction<'a, Customer>, + txn: tonbo::transaction::Transaction<'a, Customer, FoyerReader>, } impl<'db> BenchReadTransaction for TonboBenchReadTransaction<'db> { @@ -249,7 +250,7 @@ impl<'db> BenchReadTransaction for TonboBenchReadTransaction<'db> { } pub struct TonboBenchReader<'db, 'txn> { - txn: &'txn tonbo::transaction::Transaction<'db, Customer>, + txn: &'txn tonbo::transaction::Transaction<'db, Customer, FoyerReader>, } impl BenchReader for TonboBenchReader<'_, '_> { @@ -289,7 +290,7 @@ impl BenchReader for TonboBenchReader<'_, '_> { } pub struct TonboBenchWriteTransaction<'a> { - txn: tonbo::transaction::Transaction<'a, Customer>, + txn: tonbo::transaction::Transaction<'a, Customer, FoyerReader>, } impl<'db> BenchWriteTransaction for TonboBenchWriteTransaction<'db> { @@ -309,7 +310,7 @@ impl<'db> BenchWriteTransaction for TonboBenchWriteTransaction<'db> { } pub struct TonboBenchInserter<'db, 'txn> { - txn: &'txn mut tonbo::transaction::Transaction<'db, Customer>, + txn: &'txn mut tonbo::transaction::Transaction<'db, Customer, FoyerReader>, } impl BenchInserter for TonboBenchInserter<'_, '_> { diff --git a/benches/criterion/writes.rs b/benches/criterion/writes.rs index 41f3af21..d46342bb 100644 --- a/benches/criterion/writes.rs +++ b/benches/criterion/writes.rs @@ -3,6 +3,7 @@ use std::{iter::repeat_with, sync::Arc}; use criterion::{criterion_group, criterion_main, BenchmarkId, Criterion}; use mimalloc::MiMalloc; use tonbo::{executor::tokio::TokioExecutor, DbOption, Record, DB}; +use tonbo_ext_reader::foyer_reader::FoyerReader; #[global_allocator] static GLOBAL: MiMalloc = MiMalloc; @@ -15,7 +16,7 @@ pub struct KV { } #[inline(never)] -async fn tonbo_write(db: &DB, batch_size: usize) { +async fn tonbo_write(db: &DB, batch_size: usize) { let mut kvs = Vec::with_capacity(128); for _ in 0..batch_size { let key = repeat_with(fastrand::alphanumeric).take(256).collect(); From a2d10ea05398871590739a7b6d09022137bc6f0f Mon Sep 17 00:00:00 2001 From: kkould <2435992353@qq.com> Date: Fri, 1 Nov 2024 10:48:45 +0000 Subject: [PATCH 09/16] bench: add `tonbo_s3` --- Cargo.toml | 6 +-- benches/common.rs | 92 +++++++++++++++++++++++++++++++++++++ benches/read_bench.rs | 14 ++++-- benches/write_bench.rs | 12 ++++- tonbo_ext_reader/Cargo.toml | 6 +-- 5 files changed, 119 insertions(+), 11 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index f424e6a6..2cb524e8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -58,7 +58,7 @@ crc32fast = "1" crossbeam-skiplist = "0.1" datafusion = { version = "42", optional = true } flume = { version = "0.11", features = ["async"] } -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio", version = "0.2.1", features = [ +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio", version = "0.3.1", features = [ "aws", "dyn", "fs", @@ -66,11 +66,11 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-dispatch", version = "0.2.0", features = [ +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-dispatch", version = "0.2.1", features = [ "aws", "tokio", ] } -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-parquet", version = "0.2.1" } futures-core = "0.3" futures-io = "0.3" futures-util = "0.3" diff --git a/benches/common.rs b/benches/common.rs index 537da116..52894256 100644 --- a/benches/common.rs +++ b/benches/common.rs @@ -7,6 +7,7 @@ use std::{ }; use async_stream::stream; +use fusio_dispatch::FsOptions; use futures_core::Stream; use futures_util::StreamExt; use parquet::data_type::AsBytes; @@ -185,6 +186,97 @@ pub trait BenchReader { ) -> impl Stream + 'a; } +pub struct TonboS3BenchDataBase { + db: tonbo::DB, +} + +impl TonboS3BenchDataBase { + #[allow(dead_code)] + pub fn new(db: tonbo::DB) -> Self { + TonboS3BenchDataBase { db } + } +} + +impl BenchDatabase for TonboS3BenchDataBase { + type W<'db> + = TonboBenchWriteTransaction<'db> + where + Self: 'db; + type R<'db> + = TonboBenchReadTransaction<'db> + where + Self: 'db; + + fn db_type_name() -> &'static str { + "tonbo on s3" + } + + async fn write_transaction(&self) -> Self::W<'_> { + TonboBenchWriteTransaction { + txn: self.db.transaction().await, + } + } + + async fn read_transaction(&self) -> Self::R<'_> { + TonboBenchReadTransaction { + txn: self.db.transaction().await, + } + } + + async fn build(path: impl AsRef) -> Self { + create_dir_all(path.as_ref()).await.unwrap(); + + let fs_options = FsOptions::S3 { + bucket: "data".to_string(), + credential: Some(fusio::remotes::aws::credential::AwsCredential { + key_id: "user".to_string(), + secret_key: "password".to_string(), + token: None, + }), + endpoint: Some("http://localhost:9000".to_string()), + sign_payload: None, + checksum: None, + region: None, + }; + + let path = fusio::path::Path::from_filesystem_path(path.as_ref()).unwrap(); + let option = DbOption::from(path.clone()) + .level_path( + 0, + fusio::path::Path::from_url_path("/l0").unwrap(), + fs_options.clone(), + ) + .unwrap() + .level_path( + 1, + fusio::path::Path::from_url_path("/l1").unwrap(), + fs_options.clone(), + ) + .unwrap() + .level_path( + 2, + fusio::path::Path::from_url_path("/l2").unwrap(), + fs_options.clone(), + ) + .unwrap() + .level_path( + 3, + fusio::path::Path::from_url_path("/l3").unwrap(), + fs_options.clone(), + ) + .unwrap() + .level_path( + 4, + fusio::path::Path::from_url_path("/l4").unwrap(), + fs_options.clone(), + ) + .unwrap() + .disable_wal(); + + TonboS3BenchDataBase::new(tonbo::DB::new(option, TokioExecutor::new()).await.unwrap()) + } +} + pub struct TonboBenchDataBase { db: tonbo::DB, } diff --git a/benches/read_bench.rs b/benches/read_bench.rs index c849fcb9..f2099453 100644 --- a/benches/read_bench.rs +++ b/benches/read_bench.rs @@ -12,7 +12,8 @@ use tokio::{fs, io::AsyncWriteExt}; use crate::common::{ read_tbl, BenchDatabase, BenchReadTransaction, BenchReader, RedbBenchDatabase, - RocksdbBenchDatabase, SledBenchDatabase, TonboBenchDataBase, ITERATIONS, NUM_SCAN, READ_TIMES, + RocksdbBenchDatabase, SledBenchDatabase, TonboBenchDataBase, TonboS3BenchDataBase, ITERATIONS, + NUM_SCAN, READ_TIMES, }; async fn benchmark( @@ -152,10 +153,13 @@ async fn main() { load::(&tbl_path, data_dir.join("tonbo")).await; load::(&tbl_path, data_dir.join("rocksdb")).await; + load::(&tbl_path, data_dir.join("tonbo_s3")).await; } let tonbo_latency_results = { benchmark::(data_dir.join("tonbo")).await }; let rocksdb_results = { benchmark::(data_dir.join("rocksdb")).await }; + let tonbo_s3_latency_results = + { benchmark::(data_dir.join("tonbo_s3")).await }; let mut rows: Vec> = Vec::new(); @@ -163,7 +167,11 @@ async fn main() { rows.push(vec![benchmark.to_string()]); } - for results in [tonbo_latency_results, rocksdb_results] { + for results in [ + tonbo_latency_results, + rocksdb_results, + tonbo_s3_latency_results, + ] { for (i, (_benchmark, duration)) in results.iter().enumerate() { rows[i].push(format!("{}ms", duration.as_millis())); } @@ -171,7 +179,7 @@ async fn main() { let mut table = comfy_table::Table::new(); table.set_width(100); - table.set_header(["", "tonbo", "rocksdb"]); + table.set_header(["", "tonbo", "rocksdb", "tonbo_s3"]); for row in rows { table.add_row(row); } diff --git a/benches/write_bench.rs b/benches/write_bench.rs index f2f43354..2d2a3aff 100644 --- a/benches/write_bench.rs +++ b/benches/write_bench.rs @@ -201,6 +201,10 @@ async fn main() { let tmp_file: TempDir = tempfile::tempdir_in(&tmpdir).unwrap(); benchmark::(tmp_file.path()).await }; + let tonbo_s3_latency_results = { + let tmp_file: TempDir = tempfile::tempdir_in(&tmpdir).unwrap(); + benchmark::(tmp_file.path()).await + }; let _ = fs::remove_dir_all(&tmpdir); @@ -210,7 +214,11 @@ async fn main() { rows.push(vec![benchmark.to_string()]); } - for results in [tonbo_latency_results, rocksdb_results] { + for results in [ + tonbo_latency_results, + rocksdb_results, + tonbo_s3_latency_results, + ] { for (i, (_benchmark, duration)) in results.iter().enumerate() { rows[i].push(format!("{}ms", duration.as_millis())); } @@ -218,7 +226,7 @@ async fn main() { let mut table = comfy_table::Table::new(); table.set_width(100); - table.set_header(["", "tonbo", "rocksdb"]); + table.set_header(["", "tonbo", "rocksdb", "tonbo_s3"]); for row in rows { table.add_row(row); } diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml index 1e792d35..dcb011e4 100644 --- a/tonbo_ext_reader/Cargo.toml +++ b/tonbo_ext_reader/Cargo.toml @@ -11,13 +11,13 @@ bytes = { version = "1.7", features = ["serde"] } foyer = { version = "0.12" } futures-core = "0.3" futures-util = "0.3" -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-parquet", version = "0.2.0" } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-parquet", version = "0.2.1" } parquet = { version = "53", features = ["async"] } thiserror = "1" ulid = { version = "1", features = ["serde"] } [dev-dependencies] -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio", version = "0.2.1", features = [ +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio", version = "0.3.1", features = [ "aws", "dyn", "fs", @@ -25,7 +25,7 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "d2a5e7c0d17d173b1f67af25b7ff133e93244160", package = "fusio-dispatch", version = "0.2.0", features = [ +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-dispatch", version = "0.2.1", features = [ "tokio", ] } tempfile = "3" From e6521b4256e345b9282e143feb7c395f08a4e50d Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 4 Nov 2024 11:18:47 +0800 Subject: [PATCH 10/16] chore: resolve conflicts --- Cargo.toml | 8 ++++---- tests/data_integrity.rs | 2 +- tonbo_ext_reader/Cargo.toml | 6 +++--- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 2cb524e8..95ea8aff 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -58,7 +58,7 @@ crc32fast = "1" crossbeam-skiplist = "0.1" datafusion = { version = "42", optional = true } flume = { version = "0.11", features = ["async"] } -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio", version = "0.3.1", features = [ +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = [ "aws", "dyn", "fs", @@ -66,11 +66,11 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fd "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-dispatch", version = "0.2.1", features = [ +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ "aws", "tokio", ] } -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-parquet", version = "0.2.1" } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-parquet", version = "0.2.1" } futures-core = "0.3" futures-io = "0.3" futures-util = "0.3" @@ -82,7 +82,7 @@ regex = "1" thiserror = "1" tokio = { version = "1", features = ["io-util"], default-features = false } tokio-util = { version = "0.7" } -tonbo_macros = { version = "0.1.0", path = "tonbo_macros" } +tonbo_macros = { version = "0.2.0", path = "tonbo_macros" } tonbo_ext_reader = { version = "0.1.0", path = "tonbo_ext_reader" } tracing = "0.1" ulid = "1" diff --git a/tests/data_integrity.rs b/tests/data_integrity.rs index 0cd2b39f..302fd266 100644 --- a/tests/data_integrity.rs +++ b/tests/data_integrity.rs @@ -63,8 +63,8 @@ mod tests { } } - #[tokio::test] #[ignore] + #[tokio::test] async fn test_data_integrity() { let mut rng = fastrand::Rng::with_seed(42); let mut primary_key_count = 0; diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml index dcb011e4..ed1e9562 100644 --- a/tonbo_ext_reader/Cargo.toml +++ b/tonbo_ext_reader/Cargo.toml @@ -11,13 +11,13 @@ bytes = { version = "1.7", features = ["serde"] } foyer = { version = "0.12" } futures-core = "0.3" futures-util = "0.3" -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-parquet", version = "0.2.1" } +fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-parquet", version = "0.2.1" } parquet = { version = "53", features = ["async"] } thiserror = "1" ulid = { version = "1", features = ["serde"] } [dev-dependencies] -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio", version = "0.3.1", features = [ +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = [ "aws", "dyn", "fs", @@ -25,7 +25,7 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fd "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "a97407aaea1a82fde1cff997e4db13be0f24b517", package = "fusio-dispatch", version = "0.2.1", features = [ +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ "tokio", ] } tempfile = "3" From 6cce98b2fc72840b1271f34cd6665f2d1af1c5b6 Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 4 Nov 2024 11:21:40 +0800 Subject: [PATCH 11/16] ci: reduce benchmark trigger limit --- .github/workflows/ci.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index f70840ce..ac7bdba2 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -93,7 +93,6 @@ jobs: contents: write pull-requests: write repository-projects: write - if: github.event_name == 'pull_request' steps: - uses: actions/checkout@v4 - name: Install latest nightly From f24b6f178a338ebbb05a6316c348c580fbe1f7c8 Mon Sep 17 00:00:00 2001 From: Kould Date: Tue, 5 Nov 2024 15:52:26 +0800 Subject: [PATCH 12/16] feat: add `LruReader` for WASM --- benches/common.rs | 91 +++++++++--- src/compaction/mod.rs | 4 + src/ondisk/sstable.rs | 2 + src/option.rs | 4 + src/version/set.rs | 4 + tonbo_ext_reader/Cargo.toml | 2 + tonbo_ext_reader/src/foyer_reader.rs | 157 +-------------------- tonbo_ext_reader/src/lib.rs | 166 ++++++++++++++++++++++ tonbo_ext_reader/src/lru_reader.rs | 199 +++++++++++++++++++++++++++ 9 files changed, 453 insertions(+), 176 deletions(-) create mode 100644 tonbo_ext_reader/src/lru_reader.rs diff --git a/benches/common.rs b/benches/common.rs index 52894256..28dc8769 100644 --- a/benches/common.rs +++ b/benches/common.rs @@ -17,7 +17,7 @@ use tokio::fs::create_dir_all; use tonbo::{ executor::tokio::TokioExecutor, stream, transaction::TransactionEntry, DbOption, Projection, }; -use tonbo_ext_reader::foyer_reader::FoyerReader; +use tonbo_ext_reader::{foyer_reader::FoyerReader, lru_reader::LruReader, CacheReader}; use tonbo_macros::Record; const RNG_SEED: u64 = 3; @@ -199,11 +199,11 @@ impl TonboS3BenchDataBase { impl BenchDatabase for TonboS3BenchDataBase { type W<'db> - = TonboBenchWriteTransaction<'db> + = TonboBenchWriteTransaction<'db, FoyerReader> where Self: 'db; type R<'db> - = TonboBenchReadTransaction<'db> + = TonboBenchReadTransaction<'db, FoyerReader> where Self: 'db; @@ -277,24 +277,73 @@ impl BenchDatabase for TonboS3BenchDataBase { } } -pub struct TonboBenchDataBase { +pub struct TonboFoyerBenchDataBase { db: tonbo::DB, } -impl TonboBenchDataBase { +impl TonboFoyerBenchDataBase { #[allow(dead_code)] pub fn new(db: tonbo::DB) -> Self { + TonboFoyerBenchDataBase { db } + } +} + +impl BenchDatabase for TonboFoyerBenchDataBase { + type W<'db> + = TonboBenchWriteTransaction<'db, FoyerReader> + where + Self: 'db; + type R<'db> + = TonboBenchReadTransaction<'db, FoyerReader> + where + Self: 'db; + + fn db_type_name() -> &'static str { + "tonbo" + } + + async fn write_transaction(&self) -> Self::W<'_> { + TonboBenchWriteTransaction { + txn: self.db.transaction().await, + } + } + + async fn read_transaction(&self) -> Self::R<'_> { + TonboBenchReadTransaction { + txn: self.db.transaction().await, + } + } + + async fn build(path: impl AsRef) -> Self { + create_dir_all(path.as_ref()).await.unwrap(); + + let option = + DbOption::from(fusio::path::Path::from_filesystem_path(path.as_ref()).unwrap()) + .disable_wal(); + + let db = tonbo::DB::new(option, TokioExecutor::new()).await.unwrap(); + TonboFoyerBenchDataBase::new(db) + } +} + +pub struct TonboBenchDataBase { + db: tonbo::DB, +} + +impl TonboBenchDataBase { + #[allow(dead_code)] + pub fn new(db: tonbo::DB) -> Self { TonboBenchDataBase { db } } } impl BenchDatabase for TonboBenchDataBase { type W<'db> - = TonboBenchWriteTransaction<'db> + = TonboBenchWriteTransaction<'db, LruReader> where Self: 'db; type R<'db> - = TonboBenchReadTransaction<'db> + = TonboBenchReadTransaction<'db, LruReader> where Self: 'db; @@ -326,13 +375,13 @@ impl BenchDatabase for TonboBenchDataBase { } } -pub struct TonboBenchReadTransaction<'a> { - txn: tonbo::transaction::Transaction<'a, Customer, FoyerReader>, +pub struct TonboBenchReadTransaction<'a, R: CacheReader> { + txn: tonbo::transaction::Transaction<'a, Customer, R>, } -impl<'db> BenchReadTransaction for TonboBenchReadTransaction<'db> { +impl<'db, R: CacheReader + 'static> BenchReadTransaction for TonboBenchReadTransaction<'db, R> { type T<'txn> - = TonboBenchReader<'db, 'txn> + = TonboBenchReader<'db, 'txn, R> where Self: 'txn; @@ -341,11 +390,11 @@ impl<'db> BenchReadTransaction for TonboBenchReadTransaction<'db> { } } -pub struct TonboBenchReader<'db, 'txn> { - txn: &'txn tonbo::transaction::Transaction<'db, Customer, FoyerReader>, +pub struct TonboBenchReader<'db, 'txn, R: CacheReader> { + txn: &'txn tonbo::transaction::Transaction<'db, Customer, R>, } -impl BenchReader for TonboBenchReader<'_, '_> { +impl BenchReader for TonboBenchReader<'_, '_, R> { async fn get<'a>(&'a self, key: &'a ItemKey) -> Option { self.txn .get(key, Projection::All) @@ -381,13 +430,13 @@ impl BenchReader for TonboBenchReader<'_, '_> { } } -pub struct TonboBenchWriteTransaction<'a> { - txn: tonbo::transaction::Transaction<'a, Customer, FoyerReader>, +pub struct TonboBenchWriteTransaction<'a, R: CacheReader> { + txn: tonbo::transaction::Transaction<'a, Customer, R>, } -impl<'db> BenchWriteTransaction for TonboBenchWriteTransaction<'db> { +impl<'db, R: CacheReader + 'static> BenchWriteTransaction for TonboBenchWriteTransaction<'db, R> { type W<'txn> - = TonboBenchInserter<'db, 'txn> + = TonboBenchInserter<'db, 'txn, R> where Self: 'txn; @@ -401,11 +450,11 @@ impl<'db> BenchWriteTransaction for TonboBenchWriteTransaction<'db> { } } -pub struct TonboBenchInserter<'db, 'txn> { - txn: &'txn mut tonbo::transaction::Transaction<'db, Customer, FoyerReader>, +pub struct TonboBenchInserter<'db, 'txn, R: CacheReader> { + txn: &'txn mut tonbo::transaction::Transaction<'db, Customer, R>, } -impl BenchInserter for TonboBenchInserter<'_, '_> { +impl BenchInserter for TonboBenchInserter<'_, '_, R> { fn insert(&mut self, record: Customer) -> Result<(), ()> { self.txn.insert(record); Ok(()) diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 321ede51..4edbf7be 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -1077,6 +1077,8 @@ pub(crate) mod tests { option.cache_meta_ratio, option.cache_range_memory, option.cache_range_disk, + option.cache_range_capacity, + option.cache_range_shards, ) .await .unwrap(); @@ -1210,6 +1212,8 @@ pub(crate) mod tests { option.cache_meta_ratio, option.cache_range_memory, option.cache_range_disk, + option.cache_range_capacity, + option.cache_range_shards, ) .await .unwrap(); diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index 5181d62b..abb32005 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -193,6 +193,8 @@ pub(crate) mod tests { option.cache_meta_ratio, option.cache_range_memory, option.cache_range_disk, + option.cache_range_capacity, + option.cache_range_shards, ) .await .unwrap(); diff --git a/src/option.rs b/src/option.rs index 3f90a57d..7a6709fe 100644 --- a/src/option.rs +++ b/src/option.rs @@ -32,6 +32,8 @@ pub struct DbOption { pub(crate) cache_meta_ratio: f64, pub(crate) cache_range_memory: usize, pub(crate) cache_range_disk: usize, + pub(crate) cache_range_capacity: usize, + pub(crate) cache_range_shards: usize, pub(crate) clean_channel_buffer: usize, pub(crate) base_path: Path, @@ -80,6 +82,8 @@ where cache_meta_ratio: 0.1, cache_range_memory: memory, cache_range_disk: 8 * memory, + cache_range_capacity: 1024, + cache_range_shards: 32, immutable_chunk_num: 3, immutable_chunk_max_num: 5, major_threshold_with_sst_size: 4, diff --git a/src/version/set.rs b/src/version/set.rs index 8a500b7f..93d16d08 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -167,6 +167,8 @@ where option.cache_meta_ratio, option.cache_range_memory, option.cache_range_disk, + option.cache_range_capacity, + option.cache_range_shards, ) .await?; @@ -361,6 +363,8 @@ pub(crate) mod tests { option.cache_meta_ratio, option.cache_range_memory, option.cache_range_disk, + option.cache_range_capacity, + option.cache_range_shards, ) .await?; diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml index ed1e9562..1ebe3345 100644 --- a/tonbo_ext_reader/Cargo.toml +++ b/tonbo_ext_reader/Cargo.toml @@ -12,6 +12,8 @@ foyer = { version = "0.12" } futures-core = "0.3" futures-util = "0.3" fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-parquet", version = "0.2.1" } +lru = "0.12" +parking_lot = "0.12" parquet = { version = "53", features = ["async"] } thiserror = "1" ulid = { version = "1", features = ["serde"] } diff --git a/tonbo_ext_reader/src/foyer_reader.rs b/tonbo_ext_reader/src/foyer_reader.rs index 4e79f544..2411acc5 100644 --- a/tonbo_ext_reader/src/foyer_reader.rs +++ b/tonbo_ext_reader/src/foyer_reader.rs @@ -69,6 +69,8 @@ impl CacheReader for FoyerReader { cache_meta_ratio: f64, cache_range_memory: usize, cache_range_disk: usize, + _: usize, + _: usize, ) -> Result<(Self::MetaCache, Self::RangeCache), CacheError> { let meta_cache = CacheBuilder::new(cache_meta_capacity) .with_shards(cache_meta_shards) @@ -125,158 +127,3 @@ impl AsyncFileReader for FoyerReader { .boxed() } } - -#[cfg(test)] -pub(crate) mod tests { - use std::{ - fs::File, - ops::Range, - sync::{ - atomic::{AtomicUsize, Ordering::SeqCst}, - Arc, - }, - }; - - use arrow::{ - array::{BooleanArray, RecordBatch, StringArray, UInt32Array}, - datatypes::{DataType, Field, Schema}, - }; - use fusio::{ - dynamic::DynFile, fs::OpenOptions, path::Path, Error, IoBuf, IoBufMut, Read, Write, - }; - use fusio_dispatch::FsOptions; - use fusio_parquet::{reader::AsyncReader, writer::AsyncWriter}; - use parquet::arrow::{async_reader::AsyncFileReader, AsyncArrowWriter}; - use tempfile::TempDir; - use ulid::Ulid; - - use crate::{foyer_reader::FoyerReader, CacheReader}; - - struct CountFile { - inner: Box, - read_count: Arc, - } - - impl Read for CountFile { - async fn read_exact_at(&mut self, buf: B, pos: u64) -> (Result<(), Error>, B) { - self.read_count.fetch_add(1, SeqCst); - self.inner.read_exact_at(buf, pos).await - } - - async fn read_to_end_at(&mut self, buf: Vec, pos: u64) -> (Result<(), Error>, Vec) { - self.read_count.fetch_add(1, SeqCst); - self.inner.read_to_end_at(buf, pos).await - } - - async fn size(&self) -> Result { - self.inner.size().await - } - } - - impl Write for CountFile { - async fn write_all(&mut self, buf: B) -> (Result<(), Error>, B) { - self.inner.write_all(buf).await - } - - async fn flush(&mut self) -> Result<(), Error> { - self.inner.flush().await - } - - async fn close(&mut self) -> Result<(), Error> { - self.inner.close().await - } - } - - #[tokio::test] - async fn test_cache_read() { - let temp_dir = TempDir::new().unwrap(); - - let parquet_path = { - let path = temp_dir.path().join("test.parquet"); - let _ = File::create(&path).unwrap(); - - Path::from_filesystem_path(&path).unwrap() - }; - let fs = FsOptions::Local.parse().unwrap(); - let schema = Arc::new(Schema::new(vec![ - Field::new("_null", DataType::Boolean, false), - Field::new("_ts", DataType::UInt32, false), - Field::new("vstring", DataType::Utf8, false), - Field::new("vu32", DataType::UInt32, false), - Field::new("vbool", DataType::Boolean, true), - ])); - let mut writer = AsyncArrowWriter::try_new( - AsyncWriter::new( - fs.open_options( - &parquet_path, - OpenOptions::default().read(true).write(true).create(true), - ) - .await - .unwrap(), - ), - schema.clone(), - None, - ) - .unwrap(); - writer - .write( - &RecordBatch::try_new( - schema.clone(), - vec![ - Arc::new(BooleanArray::from(vec![false, false, false])), - Arc::new(UInt32Array::from(vec![0, 1, 2])), - Arc::new(StringArray::from(vec!["a", "b", "c"])), - Arc::new(UInt32Array::from(vec![0, 1, 2])), - Arc::new(BooleanArray::from(vec![true, true, true])), - ], - ) - .unwrap(), - ) - .await - .unwrap(); - writer.close().await.unwrap(); - - let read_count = Arc::new(AtomicUsize::new(0)); - let (meta_cache, range_cache) = FoyerReader::build_caches( - temp_dir.path().join("cache"), - 32, - 4, - 0.1, - 64 * 1024 * 1024, - 254 * 1024 * 1024, - ) - .await - .unwrap(); - - let gen = Ulid::new(); - for _ in 0..1000 { - let file = fs - .open_options(&parquet_path, OpenOptions::default().read(true)) - .await - .unwrap(); - let content_len = file.size().await.unwrap(); - - let mut reader = FoyerReader::new( - meta_cache.clone(), - range_cache.clone(), - gen, - AsyncReader::new( - Box::new(CountFile { - inner: file, - read_count: read_count.clone(), - }), - content_len, - ) - .await - .unwrap(), - ); - - let _ = AsyncFileReader::get_metadata(&mut reader).await.unwrap(); - let _ = AsyncFileReader::get_bytes(&mut reader, Range { start: 0, end: 10 }) - .await - .unwrap(); - } - - assert_eq!(read_count.load(SeqCst), 2); - } -} diff --git a/tonbo_ext_reader/src/lib.rs b/tonbo_ext_reader/src/lib.rs index e9cbab10..1b738d35 100644 --- a/tonbo_ext_reader/src/lib.rs +++ b/tonbo_ext_reader/src/lib.rs @@ -7,6 +7,7 @@ use thiserror::Error; use ulid::Ulid; pub mod foyer_reader; +pub mod lru_reader; pub trait MetaCache: Sync + Send + Clone + Debug { fn get(&self, gen: &Ulid) -> Option>; @@ -34,6 +35,7 @@ pub trait CacheReader: AsyncFileReader + Unpin { inner: AsyncReader, ) -> Self; + #[allow(clippy::too_many_arguments)] fn build_caches( cache_path: impl AsRef + Send, cache_meta_capacity: usize, @@ -41,6 +43,8 @@ pub trait CacheReader: AsyncFileReader + Unpin { cache_meta_ratio: f64, cache_range_memory: usize, cache_range_disk: usize, + cache_range_capacity: usize, + cache_range_shards: usize, ) -> impl std::future::Future> + Send; } @@ -51,3 +55,165 @@ pub enum CacheError { #[error("foyer error: {0}")] Foyer(#[from] anyhow::Error), } + +#[cfg(test)] +pub(crate) mod tests { + use std::{ + fs::File, + ops::Range, + sync::{ + atomic::{AtomicUsize, Ordering::SeqCst}, + Arc, + }, + }; + + use arrow::{ + array::{BooleanArray, RecordBatch, StringArray, UInt32Array}, + datatypes::{DataType, Field, Schema}, + }; + use fusio::{ + dynamic::DynFile, fs::OpenOptions, path::Path, Error, IoBuf, IoBufMut, Read, Write, + }; + use fusio_dispatch::FsOptions; + use fusio_parquet::{reader::AsyncReader, writer::AsyncWriter}; + use parquet::arrow::{async_reader::AsyncFileReader, AsyncArrowWriter}; + use tempfile::TempDir; + use ulid::Ulid; + + use crate::{foyer_reader::FoyerReader, lru_reader::LruReader, CacheReader}; + + struct CountFile { + inner: Box, + read_count: Arc, + } + + impl Read for CountFile { + async fn read_exact_at(&mut self, buf: B, pos: u64) -> (Result<(), Error>, B) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_exact_at(buf, pos).await + } + + async fn read_to_end_at(&mut self, buf: Vec, pos: u64) -> (Result<(), Error>, Vec) { + self.read_count.fetch_add(1, SeqCst); + self.inner.read_to_end_at(buf, pos).await + } + + async fn size(&self) -> Result { + self.inner.size().await + } + } + + impl Write for CountFile { + async fn write_all(&mut self, buf: B) -> (Result<(), Error>, B) { + self.inner.write_all(buf).await + } + + async fn flush(&mut self) -> Result<(), Error> { + self.inner.flush().await + } + + async fn close(&mut self) -> Result<(), Error> { + self.inner.close().await + } + } + + async fn inner_test_cache_read() { + let temp_dir = TempDir::new().unwrap(); + + let parquet_path = { + let path = temp_dir.path().join("test.parquet"); + let _ = File::create(&path).unwrap(); + + Path::from_filesystem_path(&path).unwrap() + }; + let fs = FsOptions::Local.parse().unwrap(); + let schema = Arc::new(Schema::new(vec![ + Field::new("_null", DataType::Boolean, false), + Field::new("_ts", DataType::UInt32, false), + Field::new("vstring", DataType::Utf8, false), + Field::new("vu32", DataType::UInt32, false), + Field::new("vbool", DataType::Boolean, true), + ])); + let mut writer = AsyncArrowWriter::try_new( + AsyncWriter::new( + fs.open_options( + &parquet_path, + OpenOptions::default().read(true).write(true).create(true), + ) + .await + .unwrap(), + ), + schema.clone(), + None, + ) + .unwrap(); + writer + .write( + &RecordBatch::try_new( + schema.clone(), + vec![ + Arc::new(BooleanArray::from(vec![false, false, false])), + Arc::new(UInt32Array::from(vec![0, 1, 2])), + Arc::new(StringArray::from(vec!["a", "b", "c"])), + Arc::new(UInt32Array::from(vec![0, 1, 2])), + Arc::new(BooleanArray::from(vec![true, true, true])), + ], + ) + .unwrap(), + ) + .await + .unwrap(); + writer.close().await.unwrap(); + + let read_count = Arc::new(AtomicUsize::new(0)); + let (meta_cache, range_cache) = R::build_caches( + temp_dir.path().join("cache"), + 32, + 4, + 0.1, + 64 * 1024 * 1024, + 254 * 1024 * 1024, + 128, + 16, + ) + .await + .unwrap(); + + let gen = Ulid::new(); + for _ in 0..1000 { + let file = fs + .open_options(&parquet_path, OpenOptions::default().read(true)) + .await + .unwrap(); + let content_len = file.size().await.unwrap(); + + let mut reader = R::new( + meta_cache.clone(), + range_cache.clone(), + gen, + AsyncReader::new( + Box::new(CountFile { + inner: file, + read_count: read_count.clone(), + }), + content_len, + ) + .await + .unwrap(), + ); + + let _ = AsyncFileReader::get_metadata(&mut reader).await.unwrap(); + let _ = AsyncFileReader::get_bytes(&mut reader, Range { start: 0, end: 10 }) + .await + .unwrap(); + } + + assert_eq!(read_count.load(SeqCst), 2); + } + + #[tokio::test] + async fn test_cache_read() { + inner_test_cache_read::().await; + inner_test_cache_read::().await; + } +} diff --git a/tonbo_ext_reader/src/lru_reader.rs b/tonbo_ext_reader/src/lru_reader.rs new file mode 100644 index 00000000..e3f4a5d3 --- /dev/null +++ b/tonbo_ext_reader/src/lru_reader.rs @@ -0,0 +1,199 @@ +use std::{ + hash::{BuildHasher, Hash, RandomState}, + num::NonZeroUsize, + ops::Range, + path::Path, + sync::Arc, +}; + +use bytes::Bytes; +use fusio_parquet::reader::AsyncReader; +use futures_core::future::BoxFuture; +use futures_util::FutureExt; +use lru::LruCache; +use parking_lot::Mutex; +use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; +use ulid::Ulid; + +use crate::{CacheError, CacheReader, MetaCache, RangeCache}; + +pub(crate) trait SharedKey: Hash + PartialEq + Eq { + fn shared(&self, hash_builder: &S, shared: usize) -> usize; +} + +#[derive(Debug)] +pub(crate) struct SharedLruCache +where + K: SharedKey + Clone, + V: Clone, + S: BuildHasher, +{ + caches: Vec>>, + hasher: S, +} + +impl SharedLruCache +where + K: SharedKey + Clone, + V: Clone, + S: BuildHasher, +{ + pub fn new(cap: usize, shared: usize, hasher: S) -> Self { + assert_eq!(cap % shared, 0); + + let mut caches = Vec::with_capacity(shared); + + for _ in 0..shared { + caches.push(Mutex::new(LruCache::new(NonZeroUsize::new(cap).unwrap()))); + } + + SharedLruCache { caches, hasher } + } + + #[inline] + pub fn get) -> Option>(&self, key: &K, fn_value: F) -> Option { + let mut guard = self.shard(key).lock(); + fn_value(guard.get(key)) + } + + #[inline] + pub fn put(&self, key: K, value: V) -> Option { + self.shard(&key).lock().put(key, value) + } + + fn sharding_size(&self) -> usize { + self.caches.len() + } + + fn shard(&self, key: &K) -> &Mutex> { + let pos = key.shared(&self.hasher, self.sharding_size()); + &self.caches[pos] + } +} + +impl SharedKey for Ulid { + fn shared(&self, hash_builder: &S, shared: usize) -> usize { + hash_builder.hash_one(self) as usize % shared + } +} + +// let the Range of the same Gen be sharded to the same LRU +impl SharedKey for (Ulid, Range) { + fn shared(&self, hash_builder: &S, shared: usize) -> usize { + self.0.shared(hash_builder, shared) + } +} + +#[derive(Debug, Clone)] +pub struct LruMetaCache(Arc>>); +#[derive(Debug, Clone)] +pub struct LruRangeCache(Arc), Bytes>>); + +pub struct LruReader { + gen: Ulid, + inner: AsyncReader, + range_cache: LruRangeCache, + meta_cache: LruMetaCache, +} + +impl MetaCache for LruMetaCache { + fn get(&self, gen: &Ulid) -> Option> { + self.0.get(gen, |v| v.map(Arc::clone)) + } + + fn insert(&self, gen: Ulid, data: Arc) -> Arc { + let _ = self.0.put(gen, data.clone()); + data + } +} + +impl RangeCache for LruRangeCache { + async fn get(&self, key: &(Ulid, Range)) -> Result, CacheError> { + Ok(self.0.get(key, |v| v.cloned())) + } + + fn insert(&self, key: (Ulid, Range), bytes: Bytes) -> Bytes { + let _ = self.0.put(key, bytes.clone()); + bytes + } +} + +impl AsyncFileReader for LruReader { + fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { + async move { + let key = (self.gen, range); + if let Some(bytes) = self + .range_cache + .get(&key) + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? + { + return Ok(bytes); + } + + let bytes = self.inner.get_bytes(key.1.clone()).await?; + Ok(self.range_cache.insert(key, bytes)) + } + .boxed() + } + + fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { + async move { + if let Some(meta) = self.meta_cache.get(&self.gen) { + return Ok(meta); + } + + let meta = self + .inner + .get_metadata() + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))?; + + Ok(self.meta_cache.insert(self.gen, meta)) + } + .boxed() + } +} + +impl CacheReader for LruReader { + type MetaCache = LruMetaCache; + type RangeCache = LruRangeCache; + + fn new( + meta_cache: Self::MetaCache, + range_cache: Self::RangeCache, + gen: Ulid, + inner: AsyncReader, + ) -> Self { + LruReader { + gen, + inner, + range_cache, + meta_cache, + } + } + + async fn build_caches( + _: impl AsRef + Send, + cache_meta_capacity: usize, + cache_meta_shards: usize, + _: f64, + _: usize, + _: usize, + cache_range_capacity: usize, + cache_range_shards: usize, + ) -> Result<(Self::MetaCache, Self::RangeCache), CacheError> { + let meta_cache = LruMetaCache(Arc::new(SharedLruCache::new( + cache_meta_capacity, + cache_meta_shards, + RandomState::default(), + ))); + let range_cache = LruRangeCache(Arc::new(SharedLruCache::new( + cache_range_capacity, + cache_range_shards, + RandomState::default(), + ))); + + Ok((meta_cache, range_cache)) + } +} From 5a279222f1685659a2b9e18dfe8a112b58d7b1f2 Mon Sep 17 00:00:00 2001 From: Kould Date: Tue, 5 Nov 2024 16:10:42 +0800 Subject: [PATCH 13/16] chore: sync version --- bindings/python/Cargo.toml | 4 ++-- bindings/python/src/fs.rs | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index a8b2eddd..0dd61b4b 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -9,8 +9,8 @@ crate-type = ["cdylib"] [workspace] [dependencies] -fusio = { package = "fusio", version = "0.3.1", features = ["aws", "tokio"] } -fusio-dispatch = { package = "fusio-dispatch", version = "0.2.0", features = [ +fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = ["aws", "tokio"] } +fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ "aws", "tokio", ] } diff --git a/bindings/python/src/fs.rs b/bindings/python/src/fs.rs index b9b7f730..ce4dbae8 100644 --- a/bindings/python/src/fs.rs +++ b/bindings/python/src/fs.rs @@ -59,6 +59,7 @@ impl From for fusio_dispatch::FsOptions { } => fusio_dispatch::FsOptions::S3 { bucket, credential: credential.map(fusio::remotes::aws::AwsCredential::from), + endpoint: None, region, sign_payload, checksum, From 3b5e11ad97caa99ab0de2bcc4876c0a4b584a73e Mon Sep 17 00:00:00 2001 From: Kould Date: Tue, 5 Nov 2024 16:35:21 +0800 Subject: [PATCH 14/16] chore: add lru_reader on read_bench --- benches/common.rs | 2 +- benches/read_bench.rs | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/benches/common.rs b/benches/common.rs index 28dc8769..ab613078 100644 --- a/benches/common.rs +++ b/benches/common.rs @@ -299,7 +299,7 @@ impl BenchDatabase for TonboFoyerBenchDataBase { Self: 'db; fn db_type_name() -> &'static str { - "tonbo" + "tonbo_foyer" } async fn write_transaction(&self) -> Self::W<'_> { diff --git a/benches/read_bench.rs b/benches/read_bench.rs index f2099453..8dd68d98 100644 --- a/benches/read_bench.rs +++ b/benches/read_bench.rs @@ -12,8 +12,8 @@ use tokio::{fs, io::AsyncWriteExt}; use crate::common::{ read_tbl, BenchDatabase, BenchReadTransaction, BenchReader, RedbBenchDatabase, - RocksdbBenchDatabase, SledBenchDatabase, TonboBenchDataBase, TonboS3BenchDataBase, ITERATIONS, - NUM_SCAN, READ_TIMES, + RocksdbBenchDatabase, SledBenchDatabase, TonboBenchDataBase, TonboFoyerBenchDataBase, + TonboS3BenchDataBase, ITERATIONS, NUM_SCAN, READ_TIMES, }; async fn benchmark( @@ -153,10 +153,13 @@ async fn main() { load::(&tbl_path, data_dir.join("tonbo")).await; load::(&tbl_path, data_dir.join("rocksdb")).await; + load::(&tbl_path, data_dir.join("tonbo_foyer")).await; load::(&tbl_path, data_dir.join("tonbo_s3")).await; } let tonbo_latency_results = { benchmark::(data_dir.join("tonbo")).await }; + let tonbo_foyer_latency_results = + { benchmark::(data_dir.join("tonbo_foyer")).await }; let rocksdb_results = { benchmark::(data_dir.join("rocksdb")).await }; let tonbo_s3_latency_results = { benchmark::(data_dir.join("tonbo_s3")).await }; @@ -169,6 +172,7 @@ async fn main() { for results in [ tonbo_latency_results, + tonbo_foyer_latency_results, rocksdb_results, tonbo_s3_latency_results, ] { @@ -179,7 +183,7 @@ async fn main() { let mut table = comfy_table::Table::new(); table.set_width(100); - table.set_header(["", "tonbo", "rocksdb", "tonbo_s3"]); + table.set_header(["", "tonbo", "tonbo_foyer", "rocksdb", "tonbo_s3"]); for row in rows { table.add_row(row); } From 89f5d59a8d67093c0e00b53e396a64b718f42a1c Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 11 Nov 2024 19:08:32 +0800 Subject: [PATCH 15/16] chore: optimization with review comments --- Cargo.toml | 11 ++++---- bindings/python/Cargo.toml | 4 +-- bindings/python/src/db.rs | 4 +-- bindings/python/src/transaction.rs | 18 ++++++------- rust-toolchain.toml | 2 +- src/compaction/mod.rs | 22 +++++++-------- src/inmem/mutable.rs | 6 ++--- src/lib.rs | 26 +++++++++--------- src/ondisk/sstable.rs | 14 +++++----- src/stream/mem_projection.rs | 4 +-- src/stream/merge.rs | 12 ++++----- src/stream/package.rs | 4 +-- src/transaction.rs | 10 +++---- src/version/set.rs | 10 +++---- tonbo_ext_reader/Cargo.toml | 11 +++++--- tonbo_ext_reader/src/foyer_reader.rs | 17 +++++++----- tonbo_ext_reader/src/lib.rs | 40 +++++++++++++--------------- tonbo_ext_reader/src/lru_reader.rs | 17 +++++++----- 18 files changed, 121 insertions(+), 111 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 95ea8aff..5e4520f9 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -11,13 +11,14 @@ resolver = "2" version = "0.2.0" [package.metadata] -msrv = "1.81.0" +msrv = "1.82.0" [features] -bench = ["redb", "rocksdb", "sled"] +bench = ["redb", "rocksdb", "sled", "foyer"] bytes = ["dep:bytes"] datafusion = ["dep:async-trait", "dep:datafusion"] default = ["bytes", "tokio"] +foyer = ["tonbo_ext_reader/foyer"] load_tbl = [] redb = ["dep:redb"] rocksdb = ["dep:rocksdb"] @@ -58,7 +59,7 @@ crc32fast = "1" crossbeam-skiplist = "0.1" datafusion = { version = "42", optional = true } flume = { version = "0.11", features = ["async"] } -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = [ +fusio = { package = "fusio", version = "0.3.3", features = [ "aws", "dyn", "fs", @@ -66,11 +67,11 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a9 "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ +fusio-dispatch = { package = "fusio-dispatch", version = "0.2.2", features = [ "aws", "tokio", ] } -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-parquet", version = "0.2.1" } +fusio-parquet = { package = "fusio-parquet", version = "0.2.2" } futures-core = "0.3" futures-io = "0.3" futures-util = "0.3" diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index 0dd61b4b..73659912 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -9,8 +9,8 @@ crate-type = ["cdylib"] [workspace] [dependencies] -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = ["aws", "tokio"] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ +fusio = { package = "fusio", version = "0.3.3", features = ["aws", "tokio"] } +fusio-dispatch = { package = "fusio-dispatch", version = "0.2.2", features = [ "aws", "tokio", ] } diff --git a/bindings/python/src/db.rs b/bindings/python/src/db.rs index 0af488fb..e640301b 100644 --- a/bindings/python/src/db.rs +++ b/bindings/python/src/db.rs @@ -11,7 +11,7 @@ use tonbo::{ record::{ColumnDesc, DynRecord}, DB, }; -use tonbo_ext_reader::foyer_reader::FoyerReader; +use tonbo_ext_reader::lru_reader::LruReader; use crate::{ column::Column, error::{CommitError, DbError}, @@ -27,7 +27,7 @@ type PyExecutor = TokioExecutor; pub struct TonboDB { desc: Arc>, primary_key_index: usize, - db: Arc>, + db: Arc>, } #[pymethods] diff --git a/bindings/python/src/transaction.rs b/bindings/python/src/transaction.rs index 2c41c024..15e70a97 100644 --- a/bindings/python/src/transaction.rs +++ b/bindings/python/src/transaction.rs @@ -7,7 +7,7 @@ use pyo3::{ }; use pyo3_asyncio::tokio::future_into_py; use tonbo::{record::DynRecord, transaction, Projection}; -use tonbo_ext_reader::foyer_reader::FoyerReader; +use tonbo_ext_reader::lru_reader::LruReader; use crate::{ column::Column, error::{repeated_commit_err, CommitError, DbError}, @@ -18,14 +18,14 @@ use crate::{ #[pyclass] pub struct Transaction { - txn: Option>, + txn: Option>, desc: Arc>, primary_key_index: usize, } impl Transaction { pub(crate) fn new<'txn>( - txn: transaction::Transaction<'txn, DynRecord, FoyerReader>, + txn: transaction::Transaction<'txn, DynRecord, LruReader>, desc: Arc>, ) -> Self { let primary_key_index = desc @@ -37,8 +37,8 @@ impl Transaction { Transaction { txn: Some(unsafe { transmute::< - transaction::Transaction<'txn, DynRecord, FoyerReader>, - transaction::Transaction<'static, DynRecord, FoyerReader>, + transaction::Transaction<'txn, DynRecord, LruReader>, + transaction::Transaction<'static, DynRecord, LruReader>, >(txn) }), desc, @@ -84,8 +84,8 @@ impl Transaction { let txn = self.txn.as_ref().unwrap(); let txn = unsafe { transmute::< - &transaction::Transaction<'_, DynRecord, FoyerReader>, - &'static transaction::Transaction<'_, DynRecord, FoyerReader>, + &transaction::Transaction<'_, DynRecord, LruReader>, + &'static transaction::Transaction<'_, DynRecord, LruReader>, >(txn) }; @@ -169,8 +169,8 @@ impl Transaction { let txn = self.txn.as_ref().unwrap(); let txn = unsafe { transmute::< - &transaction::Transaction<'_, DynRecord, FoyerReader>, - &'static transaction::Transaction<'_, DynRecord, FoyerReader>, + &transaction::Transaction<'_, DynRecord, LruReader>, + &'static transaction::Transaction<'_, DynRecord, LruReader>, >(txn) }; let col_desc = self.desc.get(self.primary_key_index).unwrap(); diff --git a/rust-toolchain.toml b/rust-toolchain.toml index db16da3a..704483a8 100644 --- a/rust-toolchain.toml +++ b/rust-toolchain.toml @@ -1,3 +1,3 @@ [toolchain] -channel = "1.81.0" +channel = "1.82.0" components = ["clippy", "rust-analyzer", "rustfmt"] diff --git a/src/compaction/mod.rs b/src/compaction/mod.rs index 4edbf7be..65bc4404 100644 --- a/src/compaction/mod.rs +++ b/src/compaction/mod.rs @@ -525,7 +525,7 @@ pub(crate) mod tests { use fusio_parquet::writer::AsyncWriter; use parquet::arrow::AsyncArrowWriter; use tempfile::TempDir; - use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; + use tonbo_ext_reader::{lru_reader::LruReader, CacheReader}; use crate::{ compaction::Compactor, @@ -684,7 +684,7 @@ pub(crate) mod tests { .await .unwrap(); - let scope = Compactor::::minor_compaction( + let scope = Compactor::::minor_compaction( &option, None, &vec![ @@ -748,7 +748,7 @@ pub(crate) mod tests { .await .unwrap(); - let scope = Compactor::::minor_compaction( + let scope = Compactor::::minor_compaction( &option, None, &vec![ @@ -813,7 +813,7 @@ pub(crate) mod tests { let max = 5.to_string(); let mut version_edits = Vec::new(); - Compactor::::major_compaction( + Compactor::::major_compaction( &version, &option, &min, @@ -859,7 +859,7 @@ pub(crate) mod tests { manager: &StoreManager, ) -> ( (FileId, FileId, FileId, FileId, FileId), - Version, + Version, ) { let level_0_fs = option .level_fs_path(0) @@ -1070,7 +1070,7 @@ pub(crate) mod tests { .unwrap(); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = FoyerReader::build_caches( + let (meta_cache, range_cache) = LruReader::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -1082,7 +1082,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let mut version = Version::::new( + let mut version = Version::::new( option.clone(), sender, Arc::new(AtomicU32::default()), @@ -1205,7 +1205,7 @@ pub(crate) mod tests { let option = Arc::new(option); let (sender, _) = bounded(1); - let (meta_cache, range_cache) = FoyerReader::build_caches( + let (meta_cache, range_cache) = LruReader::build_caches( path_to_local(&option.cache_path).unwrap(), option.cache_meta_capacity, option.cache_meta_shards, @@ -1217,7 +1217,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let mut version = Version::::new( + let mut version = Version::::new( option.clone(), sender, Arc::new(AtomicU32::default()), @@ -1241,7 +1241,7 @@ pub(crate) mod tests { let min = 6.to_string(); let max = 9.to_string(); - Compactor::::major_compaction( + Compactor::::major_compaction( &version, &option, &min, @@ -1270,7 +1270,7 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = + let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); for i in 5..9 { diff --git a/src/inmem/mutable.rs b/src/inmem/mutable.rs index e5a1ea3c..230baffa 100644 --- a/src/inmem/mutable.rs +++ b/src/inmem/mutable.rs @@ -5,7 +5,7 @@ use crossbeam_skiplist::{ map::{Entry, Range}, SkipMap, }; -use fusio::{buffered::BufWriter, dynamic::DynFile, DynFs}; +use fusio::{buffered::BufWriter, DynFs, DynWrite}; use ulid::Ulid; use crate::{ @@ -37,7 +37,7 @@ where R: Record, { pub(crate) data: SkipMap, Option>, - wal: Option, R>>>, + wal: Option, R>>>, pub(crate) trigger: Arc + Send + Sync>>, } @@ -61,7 +61,7 @@ where ) .await?, option.wal_buffer_size, - )) as Box; + )) as Box; wal = Some(Mutex::new(WalFile::new(file, file_id))); }; diff --git a/src/lib.rs b/src/lib.rs index 6e7261c5..818ad1dd 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -38,7 +38,7 @@ //! use tokio::fs; //! use tokio_util::bytes::Bytes; //! use tonbo::{executor::tokio::TokioExecutor, DbOption, Projection, Record, DB}; -//! use tonbo_ext_reader::foyer_reader::FoyerReader; +//! use tonbo_ext_reader::lru_reader::LruReader; //! //! // use macro to define schema of column family just like ORM //! // it provides type safety read & write API @@ -57,7 +57,7 @@ //! //! let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); //! // pluggable async runtime and I/O -//! let db: DB = +//! let db: DB = //! DB::new(options, TokioExecutor::default()).await.unwrap(); //! // insert with owned value //! db.insert(User { @@ -858,7 +858,7 @@ pub(crate) mod tests { use once_cell::sync::Lazy; use parquet::{arrow::ProjectionMask, format::SortingColumn, schema::types::ColumnPath}; use tempfile::TempDir; - use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; + use tonbo_ext_reader::{lru_reader::LruReader, CacheReader}; use tracing::error; use crate::{ @@ -1097,7 +1097,7 @@ pub(crate) mod tests { option: DbOption, executor: E, ) -> RecordBatch { - let db: DB = DB::new(option.clone(), executor).await.unwrap(); + let db: DB = DB::new(option.clone(), executor).await.unwrap(); let base_fs = db.manager.base_fs(); db.write( @@ -1534,7 +1534,7 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(/* max_mutable_len */ 5); - let db: DB = + let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); for (i, item) in test_items().into_iter().enumerate() { @@ -1571,7 +1571,7 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(/* max_mutable_len */ 50); - let db: DB = + let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); for item in &test_items()[0..10] { @@ -1621,7 +1621,7 @@ pub(crate) mod tests { schema.flush_wal().await.unwrap(); drop(schema); - let db: DB = + let db: DB = DB::new(option.as_ref().to_owned(), TokioExecutor::new()) .await .unwrap(); @@ -1694,7 +1694,7 @@ pub(crate) mod tests { "id".to_owned(), primary_key_index, ); - let db: DB = + let db: DB = DB::with_schema(option, TokioExecutor::new(), desc, primary_key_index) .await .unwrap(); @@ -1734,7 +1734,7 @@ pub(crate) mod tests { option.major_threshold_with_sst_size = 3; option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = + let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); for (idx, item) in test_items().into_iter().enumerate() { @@ -1777,7 +1777,7 @@ pub(crate) mod tests { option.major_default_oldest_table_num = 1; option.trigger_type = TriggerType::Length(5); - let db: DB = + let db: DB = DB::with_schema(option, TokioExecutor::new(), cols_desc, primary_key_index) .await .unwrap(); @@ -2007,7 +2007,7 @@ pub(crate) mod tests { option3.major_default_oldest_table_num = 1; option3.trigger_type = TriggerType::Length(5); - let db1: DB = DB::with_schema( + let db1: DB = DB::with_schema( option, TokioExecutor::new(), cols_desc.clone(), @@ -2015,7 +2015,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let db2: DB = DB::with_schema( + let db2: DB = DB::with_schema( option2, TokioExecutor::new(), cols_desc.clone(), @@ -2023,7 +2023,7 @@ pub(crate) mod tests { ) .await .unwrap(); - let db3: DB = + let db3: DB = DB::with_schema(option3, TokioExecutor::new(), cols_desc, primary_key_index) .await .unwrap(); diff --git a/src/ondisk/sstable.rs b/src/ondisk/sstable.rs index abb32005..8d0f5cdc 100644 --- a/src/ondisk/sstable.rs +++ b/src/ondisk/sstable.rs @@ -132,7 +132,7 @@ pub(crate) mod tests { basic::{Compression, ZstdLevel}, file::properties::WriterProperties, }; - use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; + use tonbo_ext_reader::{lru_reader::LruReader, CacheReader}; use super::SsTable; use crate::{ @@ -234,7 +234,7 @@ pub(crate) mod tests { { let test_ref_1 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .get( key.borrow(), @@ -252,7 +252,7 @@ pub(crate) mod tests { } { let test_ref_2 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .get( key.borrow(), @@ -270,7 +270,7 @@ pub(crate) mod tests { } { let test_ref_3 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .get( key.borrow(), @@ -310,7 +310,7 @@ pub(crate) mod tests { { let mut test_ref_1 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .scan( (Bound::Unbounded, Bound::Unbounded), @@ -336,7 +336,7 @@ pub(crate) mod tests { } { let mut test_ref_2 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .scan( (Bound::Unbounded, Bound::Unbounded), @@ -362,7 +362,7 @@ pub(crate) mod tests { } { let mut test_ref_3 = - open_sstable::(base_fs, table_path.clone(), table_gen, &option) + open_sstable::(base_fs, table_path.clone(), table_gen, &option) .await .scan( (Bound::Unbounded, Bound::Unbounded), diff --git a/src/stream/mem_projection.rs b/src/stream/mem_projection.rs index 92062940..230cc8be 100644 --- a/src/stream/mem_projection.rs +++ b/src/stream/mem_projection.rs @@ -68,7 +68,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; use parquet::arrow::{arrow_to_parquet_schema, ProjectionMask}; - use tonbo_ext_reader::foyer_reader::FoyerReader; + use tonbo_ext_reader::lru_reader::LruReader; use crate::{ inmem::mutable::Mutable, record::Record, stream::mem_projection::MemProjectionStream, @@ -129,7 +129,7 @@ mod tests { vec![0, 1, 2, 4], ); - let mut stream = MemProjectionStream::::new( + let mut stream = MemProjectionStream::::new( mutable .scan((Bound::Unbounded, Bound::Unbounded), 6.into()) .into(), diff --git a/src/stream/merge.rs b/src/stream/merge.rs index 7d144a6f..f086d80e 100644 --- a/src/stream/merge.rs +++ b/src/stream/merge.rs @@ -164,7 +164,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; - use tonbo_ext_reader::foyer_reader::FoyerReader; + use tonbo_ext_reader::lru_reader::LruReader; use super::MergeStream; use crate::{ @@ -217,7 +217,7 @@ mod tests { let lower = "a".to_string(); let upper = "e".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![ m1.scan(bound, 6.into()).into(), m2.scan(bound, 6.into()).into(), @@ -296,7 +296,7 @@ mod tests { let lower = "1".to_string(); let upper = "4".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1.scan(bound, 0.into()).into()], 0.into(), ) @@ -326,7 +326,7 @@ mod tests { let lower = "1".to_string(); let upper = "4".to_string(); let bound = (Bound::Included(&lower), Bound::Included(&upper)); - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1.scan(bound, 1.into()).into()], 1.into(), ) @@ -380,7 +380,7 @@ mod tests { let lower = "1".to_string(); let upper = "3".to_string(); { - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Included(&lower), Bound::Included(&upper)), 0.into()) .into()], @@ -400,7 +400,7 @@ mod tests { assert!(merge.next().await.is_none()); } { - let mut merge = MergeStream::::from_vec( + let mut merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Included(&lower), Bound::Included(&upper)), 0.into()) .into()], diff --git a/src/stream/package.rs b/src/stream/package.rs index e1a4f20d..5c63f782 100644 --- a/src/stream/package.rs +++ b/src/stream/package.rs @@ -89,7 +89,7 @@ mod tests { use fusio::{disk::TokioFs, path::Path, DynFs}; use futures_util::StreamExt; use tempfile::TempDir; - use tonbo_ext_reader::foyer_reader::FoyerReader; + use tonbo_ext_reader::lru_reader::LruReader; use crate::{ inmem::{ @@ -182,7 +182,7 @@ mod tests { .await .unwrap(); - let merge = MergeStream::::from_vec( + let merge = MergeStream::::from_vec( vec![m1 .scan((Bound::Unbounded, Bound::Unbounded), 6.into()) .into()], diff --git a/src/transaction.rs b/src/transaction.rs index 8e85b340..3e9333df 100644 --- a/src/transaction.rs +++ b/src/transaction.rs @@ -260,7 +260,7 @@ mod tests { use fusio_dispatch::FsOptions; use futures_util::StreamExt; use tempfile::TempDir; - use tonbo_ext_reader::foyer_reader::FoyerReader; + use tonbo_ext_reader::lru_reader::LruReader; use crate::{ compaction::tests::build_version, @@ -280,7 +280,7 @@ mod tests { async fn transaction_read_write() { let temp_dir = TempDir::new().unwrap(); - let db = DB::::new( + let db = DB::::new( DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()), TokioExecutor::new(), ) @@ -408,7 +408,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db = DB::::new(option, TokioExecutor::new()) + let db = DB::::new(option, TokioExecutor::new()) .await .unwrap(); @@ -441,7 +441,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db = DB::::new(option, TokioExecutor::new()) + let db = DB::::new(option, TokioExecutor::new()) .await .unwrap(); @@ -811,7 +811,7 @@ mod tests { "age".to_string(), 0, ); - let db = DB::<_, _, FoyerReader>::with_schema(option, TokioExecutor::default(), descs, 0) + let db = DB::<_, _, LruReader>::with_schema(option, TokioExecutor::default(), descs, 0) .await .unwrap(); diff --git a/src/version/set.rs b/src/version/set.rs index 93d16d08..05a9cf89 100644 --- a/src/version/set.rs +++ b/src/version/set.rs @@ -322,7 +322,7 @@ pub(crate) mod tests { use fusio_dispatch::FsOptions; use futures_util::StreamExt; use tempfile::TempDir; - use tonbo_ext_reader::{foyer_reader::FoyerReader, CacheReader}; + use tonbo_ext_reader::{lru_reader::LruReader, CacheReader}; use crate::{ fs::{manager::StoreManager, FileId, FileType}, @@ -396,7 +396,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager.clone()) .await .unwrap(); @@ -412,7 +412,7 @@ pub(crate) mod tests { drop(version_set); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager) .await .unwrap(); @@ -434,7 +434,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager.clone()) .await .unwrap(); @@ -562,7 +562,7 @@ pub(crate) mod tests { .await .unwrap(); - let version_set: VersionSet = + let version_set: VersionSet = VersionSet::new(sender.clone(), option.clone(), manager) .await .unwrap(); diff --git a/tonbo_ext_reader/Cargo.toml b/tonbo_ext_reader/Cargo.toml index 1ebe3345..5b9bb4b2 100644 --- a/tonbo_ext_reader/Cargo.toml +++ b/tonbo_ext_reader/Cargo.toml @@ -4,6 +4,9 @@ name = "tonbo_ext_reader" version = "0.1.0" edition = "2021" +[features] +foyer = [] + [dependencies] anyhow = "1" arrow = "53" @@ -11,7 +14,7 @@ bytes = { version = "1.7", features = ["serde"] } foyer = { version = "0.12" } futures-core = "0.3" futures-util = "0.3" -fusio-parquet = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-parquet", version = "0.2.1" } +fusio-parquet = { package = "fusio-parquet", version = "0.2.2" } lru = "0.12" parking_lot = "0.12" parquet = { version = "53", features = ["async"] } @@ -19,7 +22,7 @@ thiserror = "1" ulid = { version = "1", features = ["serde"] } [dev-dependencies] -fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio", version = "0.3.1", features = [ +fusio = { package = "fusio", version = "0.3.3", features = [ "aws", "dyn", "fs", @@ -27,8 +30,8 @@ fusio = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a9 "tokio", "tokio-http", ] } -fusio-dispatch = { git = "https://github.com/tonbo-io/fusio.git", rev = "db305599ae88c7a934ab1ab235d8b3efda95a242", package = "fusio-dispatch", version = "0.2.1", features = [ +fusio-dispatch = { package = "fusio-dispatch", version = "0.2.2", features = [ "tokio", ] } tempfile = "3" -tokio = { version = "1", features = ["full"] } \ No newline at end of file +tokio = { version = "1", features = ["full"] } diff --git a/tonbo_ext_reader/src/foyer_reader.rs b/tonbo_ext_reader/src/foyer_reader.rs index 2411acc5..70090bad 100644 --- a/tonbo_ext_reader/src/foyer_reader.rs +++ b/tonbo_ext_reader/src/foyer_reader.rs @@ -10,7 +10,7 @@ use futures_util::FutureExt; use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; use ulid::Ulid; -use crate::{CacheError, CacheReader, MetaCache, RangeCache}; +use crate::{CacheError, CacheReader, TonboCache}; #[derive(Debug, Clone)] pub struct FoyerMetaCache(Cache>); @@ -24,9 +24,9 @@ pub struct FoyerReader { meta_cache: FoyerMetaCache, } -impl MetaCache for FoyerMetaCache { - fn get(&self, gen: &Ulid) -> Option> { - self.0.get(gen).map(|entry| entry.value().clone()) +impl TonboCache> for FoyerMetaCache { + async fn get(&self, gen: &Ulid) -> Result>, CacheError> { + Ok(self.0.get(gen).map(|entry| entry.value().clone())) } fn insert(&self, gen: Ulid, data: Arc) -> Arc { @@ -34,7 +34,7 @@ impl MetaCache for FoyerMetaCache { } } -impl RangeCache for FoyerRangeCache { +impl TonboCache<(Ulid, Range), Bytes> for FoyerRangeCache { async fn get(&self, key: &(Ulid, Range)) -> Result, CacheError> { Ok(self.0.get(key).await?.map(|entry| entry.value().clone())) } @@ -112,7 +112,12 @@ impl AsyncFileReader for FoyerReader { fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { async move { - if let Some(meta) = self.meta_cache.get(&self.gen) { + if let Some(meta) = self + .meta_cache + .get(&self.gen) + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? + { return Ok(meta); } diff --git a/tonbo_ext_reader/src/lib.rs b/tonbo_ext_reader/src/lib.rs index 1b738d35..81fdfbd5 100644 --- a/tonbo_ext_reader/src/lib.rs +++ b/tonbo_ext_reader/src/lib.rs @@ -6,27 +6,22 @@ use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaD use thiserror::Error; use ulid::Ulid; +#[cfg(feature = "foyer")] pub mod foyer_reader; pub mod lru_reader; -pub trait MetaCache: Sync + Send + Clone + Debug { - fn get(&self, gen: &Ulid) -> Option>; - - fn insert(&self, gen: Ulid, data: Arc) -> Arc; -} - -pub trait RangeCache: Sync + Send + Clone + Debug { +pub trait TonboCache: Sync + Send + Clone + Debug { fn get( &self, - key: &(Ulid, Range), - ) -> impl std::future::Future, CacheError>> + Send; + key: &K, + ) -> impl std::future::Future, CacheError>> + Send; - fn insert(&self, key: (Ulid, Range), bytes: Bytes) -> Bytes; + fn insert(&self, key: K, value: V) -> V; } pub trait CacheReader: AsyncFileReader + Unpin { - type MetaCache: MetaCache; - type RangeCache: RangeCache; + type MetaCache: TonboCache>; + type RangeCache: TonboCache<(Ulid, Range), Bytes>; fn new( meta_cache: Self::MetaCache, @@ -37,14 +32,14 @@ pub trait CacheReader: AsyncFileReader + Unpin { #[allow(clippy::too_many_arguments)] fn build_caches( - cache_path: impl AsRef + Send, - cache_meta_capacity: usize, - cache_meta_shards: usize, - cache_meta_ratio: f64, - cache_range_memory: usize, - cache_range_disk: usize, - cache_range_capacity: usize, - cache_range_shards: usize, + path: impl AsRef + Send, + meta_capacity: usize, + meta_shards: usize, + meta_ratio: f64, + range_memory: usize, + range_disk: usize, + range_capacity: usize, + range_shards: usize, ) -> impl std::future::Future> + Send; } @@ -80,7 +75,7 @@ pub(crate) mod tests { use tempfile::TempDir; use ulid::Ulid; - use crate::{foyer_reader::FoyerReader, lru_reader::LruReader, CacheReader}; + use crate::{lru_reader::LruReader, CacheReader}; struct CountFile { inner: Box, @@ -213,7 +208,8 @@ pub(crate) mod tests { #[tokio::test] async fn test_cache_read() { - inner_test_cache_read::().await; + #[cfg(feature = "foyer")] + inner_test_cache_read::().await; inner_test_cache_read::().await; } } diff --git a/tonbo_ext_reader/src/lru_reader.rs b/tonbo_ext_reader/src/lru_reader.rs index e3f4a5d3..4a2cbc77 100644 --- a/tonbo_ext_reader/src/lru_reader.rs +++ b/tonbo_ext_reader/src/lru_reader.rs @@ -15,7 +15,7 @@ use parking_lot::Mutex; use parquet::{arrow::async_reader::AsyncFileReader, file::metadata::ParquetMetaData}; use ulid::Ulid; -use crate::{CacheError, CacheReader, MetaCache, RangeCache}; +use crate::{CacheError, CacheReader, TonboCache}; pub(crate) trait SharedKey: Hash + PartialEq + Eq { fn shared(&self, hash_builder: &S, shared: usize) -> usize; @@ -96,9 +96,9 @@ pub struct LruReader { meta_cache: LruMetaCache, } -impl MetaCache for LruMetaCache { - fn get(&self, gen: &Ulid) -> Option> { - self.0.get(gen, |v| v.map(Arc::clone)) +impl TonboCache> for LruMetaCache { + async fn get(&self, gen: &Ulid) -> Result>, CacheError> { + Ok(self.0.get(gen, |v| v.map(Arc::clone))) } fn insert(&self, gen: Ulid, data: Arc) -> Arc { @@ -107,7 +107,7 @@ impl MetaCache for LruMetaCache { } } -impl RangeCache for LruRangeCache { +impl TonboCache<(Ulid, Range), Bytes> for LruRangeCache { async fn get(&self, key: &(Ulid, Range)) -> Result, CacheError> { Ok(self.0.get(key, |v| v.cloned())) } @@ -139,7 +139,12 @@ impl AsyncFileReader for LruReader { fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { async move { - if let Some(meta) = self.meta_cache.get(&self.gen) { + if let Some(meta) = self + .meta_cache + .get(&self.gen) + .await + .map_err(|e| parquet::errors::ParquetError::External(From::from(e)))? + { return Ok(meta); } From 7ebba628e4c41a1ff2b50b1d1677ac054937f7ee Mon Sep 17 00:00:00 2001 From: Kould Date: Mon, 11 Nov 2024 19:13:52 +0800 Subject: [PATCH 16/16] chore: fix example --- examples/datafusion.rs | 11 ++++------- examples/declare.rs | 4 ++-- tests/data_integrity.rs | 4 ++-- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/examples/datafusion.rs b/examples/datafusion.rs index db571879..f9e48cad 100644 --- a/examples/datafusion.rs +++ b/examples/datafusion.rs @@ -30,7 +30,7 @@ use tokio::fs; use tonbo::{ executor::tokio::TokioExecutor, inmem::immutable::ArrowArrays, record::Record, DbOption, DB, }; -use tonbo_ext_reader::foyer_reader::FoyerReader; +use tonbo_ext_reader::lru_reader::LruReader; use tonbo_macros::Record; #[derive(Record, Debug)] @@ -42,12 +42,12 @@ pub struct Music { } struct MusicProvider { - db: Arc>, + db: Arc>, } struct MusicExec { cache: PlanProperties, - db: Arc>, + db: Arc>, projection: Option>, limit: Option, range: (Bound<::Key>, Bound<::Key>), @@ -96,10 +96,7 @@ impl TableProvider for MusicProvider { } impl MusicExec { - fn new( - db: Arc>, - projection: Option<&Vec>, - ) -> Self { + fn new(db: Arc>, projection: Option<&Vec>) -> Self { let schema = Music::arrow_schema(); let schema = if let Some(projection) = &projection { Arc::new(schema.project(projection).unwrap()) diff --git a/examples/declare.rs b/examples/declare.rs index 0a057b46..f87aeed9 100644 --- a/examples/declare.rs +++ b/examples/declare.rs @@ -5,7 +5,7 @@ use fusio::path::Path; use futures_util::stream::StreamExt; use tokio::fs; use tonbo::{executor::tokio::TokioExecutor, DbOption, Projection, Record, DB}; -use tonbo_ext_reader::foyer_reader::FoyerReader; +use tonbo_ext_reader::lru_reader::LruReader; /// Use macro to define schema of column family just like ORM /// It provides type-safe read & write API @@ -25,7 +25,7 @@ async fn main() { let options = DbOption::from(Path::from_filesystem_path("./db_path/users").unwrap()); // pluggable async runtime and I/O - let db = DB::<_, _, FoyerReader>::new(options, TokioExecutor::default()) + let db = DB::<_, _, LruReader>::new(options, TokioExecutor::default()) .await .unwrap(); diff --git a/tests/data_integrity.rs b/tests/data_integrity.rs index 302fd266..0e59c62e 100644 --- a/tests/data_integrity.rs +++ b/tests/data_integrity.rs @@ -6,7 +6,7 @@ mod tests { use futures_util::StreamExt; use tempfile::TempDir; use tonbo::{executor::tokio::TokioExecutor, DbOption, Record, DB}; - use tonbo_ext_reader::foyer_reader::FoyerReader; + use tonbo_ext_reader::lru_reader::LruReader; const WRITE_TIMES: usize = 500_000; const STRING_SIZE: usize = 50; @@ -73,7 +73,7 @@ mod tests { let temp_dir = TempDir::new().unwrap(); let option = DbOption::from(Path::from_filesystem_path(temp_dir.path()).unwrap()); - let db: DB = + let db: DB = DB::new(option, TokioExecutor::new()).await.unwrap(); for _ in 0..WRITE_TIMES {