use std::sync::Arc;
use std::time::Duration;
use common_base::readable_size::ReadableSize;
use common_telemetry::{debug, info};
use futures::AsyncWriteExt;
use object_store::ObjectStore;
use snafu::ResultExt;
use store_api::storage::RegionId;
use crate::access_layer::{
new_fs_cache_store, FilePathProvider, RegionFilePathFactory, SstInfoArray, SstWriteRequest,
WriteCachePathProvider,
};
use crate::cache::file_cache::{FileCache, FileCacheRef, FileType, IndexKey, IndexValue};
use crate::error::{self, Result};
use crate::metrics::{
FLUSH_ELAPSED, UPLOAD_BYTES_TOTAL, WRITE_CACHE_DOWNLOAD_BYTES_TOTAL,
WRITE_CACHE_DOWNLOAD_ELAPSED,
};
use crate::sst::index::intermediate::IntermediateManager;
use crate::sst::index::puffin_manager::PuffinManagerFactory;
use crate::sst::index::IndexerBuilderImpl;
use crate::sst::parquet::writer::ParquetWriter;
use crate::sst::parquet::WriteOptions;
use crate::sst::{DEFAULT_WRITE_BUFFER_SIZE, DEFAULT_WRITE_CONCURRENCY};
pub struct WriteCache {
file_cache: FileCacheRef,
puffin_manager_factory: PuffinManagerFactory,
intermediate_manager: IntermediateManager,
}
pub type WriteCacheRef = Arc<WriteCache>;
impl WriteCache {
pub async fn new(
local_store: ObjectStore,
cache_capacity: ReadableSize,
ttl: Option<Duration>,
puffin_manager_factory: PuffinManagerFactory,
intermediate_manager: IntermediateManager,
) -> Result<Self> {
let file_cache = Arc::new(FileCache::new(local_store, cache_capacity, ttl));
file_cache.recover(false).await;
Ok(Self {
file_cache,
puffin_manager_factory,
intermediate_manager,
})
}
pub async fn new_fs(
cache_dir: &str,
cache_capacity: ReadableSize,
ttl: Option<Duration>,
puffin_manager_factory: PuffinManagerFactory,
intermediate_manager: IntermediateManager,
) -> Result<Self> {
info!("Init write cache on {cache_dir}, capacity: {cache_capacity}");
let local_store = new_fs_cache_store(cache_dir).await?;
Self::new(
local_store,
cache_capacity,
ttl,
puffin_manager_factory,
intermediate_manager,
)
.await
}
pub(crate) fn file_cache(&self) -> FileCacheRef {
self.file_cache.clone()
}
pub(crate) async fn write_and_upload_sst(
&self,
write_request: SstWriteRequest,
upload_request: SstUploadRequest,
write_opts: &WriteOptions,
) -> Result<SstInfoArray> {
let timer = FLUSH_ELAPSED
.with_label_values(&["write_sst"])
.start_timer();
let region_id = write_request.metadata.region_id;
let store = self.file_cache.local_store();
let path_provider = WriteCachePathProvider::new(region_id, self.file_cache.clone());
let indexer = IndexerBuilderImpl {
op_type: write_request.op_type,
metadata: write_request.metadata.clone(),
row_group_size: write_opts.row_group_size,
puffin_manager: self
.puffin_manager_factory
.build(store, path_provider.clone()),
intermediate_manager: self.intermediate_manager.clone(),
index_options: write_request.index_options,
inverted_index_config: write_request.inverted_index_config,
fulltext_index_config: write_request.fulltext_index_config,
bloom_filter_index_config: write_request.bloom_filter_index_config,
};
let mut writer = ParquetWriter::new_with_object_store(
self.file_cache.local_store(),
write_request.metadata,
indexer,
path_provider,
)
.await;
let sst_info = writer
.write_all(write_request.source, write_request.max_sequence, write_opts)
.await?;
timer.stop_and_record();
if sst_info.is_empty() {
return Ok(sst_info);
}
let mut upload_tracker = UploadTracker::new(region_id);
let mut err = None;
let remote_store = &upload_request.remote_store;
for sst in &sst_info {
let parquet_key = IndexKey::new(region_id, sst.file_id, FileType::Parquet);
let parquet_path = upload_request
.dest_path_provider
.build_sst_file_path(sst.file_id);
if let Err(e) = self.upload(parquet_key, &parquet_path, remote_store).await {
err = Some(e);
break;
}
upload_tracker.push_uploaded_file(parquet_path);
if sst.index_metadata.file_size > 0 {
let puffin_key = IndexKey::new(region_id, sst.file_id, FileType::Puffin);
let puffin_path = upload_request
.dest_path_provider
.build_index_file_path(sst.file_id);
if let Err(e) = self.upload(puffin_key, &puffin_path, remote_store).await {
err = Some(e);
break;
}
upload_tracker.push_uploaded_file(puffin_path);
}
}
if let Some(err) = err {
upload_tracker
.clean(&sst_info, &self.file_cache, remote_store)
.await;
return Err(err);
}
Ok(sst_info)
}
pub(crate) async fn remove(&self, index_key: IndexKey) {
self.file_cache.remove(index_key).await
}
pub(crate) async fn download(
&self,
index_key: IndexKey,
remote_path: &str,
remote_store: &ObjectStore,
file_size: u64,
) -> Result<()> {
const DOWNLOAD_READER_CONCURRENCY: usize = 8;
const DOWNLOAD_READER_CHUNK_SIZE: ReadableSize = ReadableSize::mb(8);
let file_type = index_key.file_type;
let timer = WRITE_CACHE_DOWNLOAD_ELAPSED
.with_label_values(&[match file_type {
FileType::Parquet => "download_parquet",
FileType::Puffin => "download_puffin",
}])
.start_timer();
let reader = remote_store
.reader_with(remote_path)
.concurrent(DOWNLOAD_READER_CONCURRENCY)
.chunk(DOWNLOAD_READER_CHUNK_SIZE.as_bytes() as usize)
.await
.context(error::OpenDalSnafu)?
.into_futures_async_read(0..file_size)
.await
.context(error::OpenDalSnafu)?;
let cache_path = self.file_cache.cache_file_path(index_key);
let mut writer = self
.file_cache
.local_store()
.writer(&cache_path)
.await
.context(error::OpenDalSnafu)?
.into_futures_async_write();
let region_id = index_key.region_id;
let file_id = index_key.file_id;
let bytes_written =
futures::io::copy(reader, &mut writer)
.await
.context(error::DownloadSnafu {
region_id,
file_id,
file_type,
})?;
writer.close().await.context(error::DownloadSnafu {
region_id,
file_id,
file_type,
})?;
WRITE_CACHE_DOWNLOAD_BYTES_TOTAL.inc_by(bytes_written);
let elapsed = timer.stop_and_record();
debug!(
"Successfully download file '{}' to local '{}', file size: {}, region: {}, cost: {:?}s",
remote_path, cache_path, bytes_written, region_id, elapsed,
);
let index_value = IndexValue {
file_size: bytes_written as _,
};
self.file_cache.put(index_key, index_value).await;
Ok(())
}
async fn upload(
&self,
index_key: IndexKey,
upload_path: &str,
remote_store: &ObjectStore,
) -> Result<()> {
let region_id = index_key.region_id;
let file_id = index_key.file_id;
let file_type = index_key.file_type;
let cache_path = self.file_cache.cache_file_path(index_key);
let timer = FLUSH_ELAPSED
.with_label_values(&[match file_type {
FileType::Parquet => "upload_parquet",
FileType::Puffin => "upload_puffin",
}])
.start_timer();
let cached_value = self
.file_cache
.local_store()
.stat(&cache_path)
.await
.context(error::OpenDalSnafu)?;
let reader = self
.file_cache
.local_store()
.reader(&cache_path)
.await
.context(error::OpenDalSnafu)?
.into_futures_async_read(0..cached_value.content_length())
.await
.context(error::OpenDalSnafu)?;
let mut writer = remote_store
.writer_with(upload_path)
.chunk(DEFAULT_WRITE_BUFFER_SIZE.as_bytes() as usize)
.concurrent(DEFAULT_WRITE_CONCURRENCY)
.await
.context(error::OpenDalSnafu)?
.into_futures_async_write();
let bytes_written =
futures::io::copy(reader, &mut writer)
.await
.context(error::UploadSnafu {
region_id,
file_id,
file_type,
})?;
writer.close().await.context(error::UploadSnafu {
region_id,
file_id,
file_type,
})?;
UPLOAD_BYTES_TOTAL.inc_by(bytes_written);
debug!(
"Successfully upload file to remote, region: {}, file: {}, upload_path: {}, cost: {:?}s",
region_id,
file_id,
upload_path,
timer.stop_and_record()
);
let index_value = IndexValue {
file_size: bytes_written as _,
};
self.file_cache.put(index_key, index_value).await;
Ok(())
}
}
pub struct SstUploadRequest {
pub dest_path_provider: RegionFilePathFactory,
pub remote_store: ObjectStore,
}
struct UploadTracker {
region_id: RegionId,
files_uploaded: Vec<String>,
}
impl UploadTracker {
fn new(region_id: RegionId) -> Self {
Self {
region_id,
files_uploaded: Vec::new(),
}
}
fn push_uploaded_file(&mut self, path: String) {
self.files_uploaded.push(path);
}
async fn clean(
&self,
sst_info: &SstInfoArray,
file_cache: &FileCacheRef,
remote_store: &ObjectStore,
) {
common_telemetry::info!(
"Start cleaning files on upload failure, region: {}, num_ssts: {}",
self.region_id,
sst_info.len()
);
for sst in sst_info {
let parquet_key = IndexKey::new(self.region_id, sst.file_id, FileType::Parquet);
file_cache.remove(parquet_key).await;
if sst.index_metadata.file_size > 0 {
let puffin_key = IndexKey::new(self.region_id, sst.file_id, FileType::Puffin);
file_cache.remove(puffin_key).await;
}
}
for file_path in &self.files_uploaded {
if let Err(e) = remote_store.delete(file_path).await {
common_telemetry::error!(e; "Failed to delete file {}", file_path);
}
}
}
}
#[cfg(test)]
mod tests {
use common_test_util::temp_dir::create_temp_dir;
use super::*;
use crate::access_layer::OperationType;
use crate::cache::test_util::new_fs_store;
use crate::cache::{CacheManager, CacheStrategy};
use crate::region::options::IndexOptions;
use crate::sst::parquet::reader::ParquetReaderBuilder;
use crate::test_util::sst_util::{
assert_parquet_metadata_eq, new_batch_by_range, new_source, sst_file_handle_with_file_id,
sst_region_metadata,
};
use crate::test_util::TestEnv;
#[tokio::test]
async fn test_write_and_upload_sst() {
let mut env = TestEnv::new();
let mock_store = env.init_object_store_manager();
let path_provider = RegionFilePathFactory::new("test".to_string());
let local_dir = create_temp_dir("");
let local_store = new_fs_store(local_dir.path().to_str().unwrap());
let write_cache = env
.create_write_cache(local_store.clone(), ReadableSize::mb(10))
.await;
let metadata = Arc::new(sst_region_metadata());
let region_id = metadata.region_id;
let source = new_source(&[
new_batch_by_range(&["a", "d"], 0, 60),
new_batch_by_range(&["b", "f"], 0, 40),
new_batch_by_range(&["b", "h"], 100, 200),
]);
let write_request = SstWriteRequest {
op_type: OperationType::Flush,
metadata,
source,
storage: None,
max_sequence: None,
cache_manager: Default::default(),
index_options: IndexOptions::default(),
inverted_index_config: Default::default(),
fulltext_index_config: Default::default(),
bloom_filter_index_config: Default::default(),
};
let upload_request = SstUploadRequest {
dest_path_provider: path_provider.clone(),
remote_store: mock_store.clone(),
};
let write_opts = WriteOptions {
row_group_size: 512,
..Default::default()
};
let sst_info = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts)
.await
.unwrap()
.remove(0); let file_id = sst_info.file_id;
let sst_upload_path = path_provider.build_sst_file_path(file_id);
let index_upload_path = path_provider.build_index_file_path(file_id);
let key = IndexKey::new(region_id, file_id, FileType::Parquet);
assert!(write_cache.file_cache.contains_key(&key));
let remote_data = mock_store.read(&sst_upload_path).await.unwrap();
let cache_data = local_store
.read(&write_cache.file_cache.cache_file_path(key))
.await
.unwrap();
assert_eq!(remote_data.to_vec(), cache_data.to_vec());
let index_key = IndexKey::new(region_id, file_id, FileType::Puffin);
assert!(write_cache.file_cache.contains_key(&index_key));
let remote_index_data = mock_store.read(&index_upload_path).await.unwrap();
let cache_index_data = local_store
.read(&write_cache.file_cache.cache_file_path(index_key))
.await
.unwrap();
assert_eq!(remote_index_data.to_vec(), cache_index_data.to_vec());
let sst_index_key = IndexKey::new(region_id, file_id, FileType::Parquet);
write_cache.remove(sst_index_key).await;
assert!(!write_cache.file_cache.contains_key(&sst_index_key));
write_cache.remove(index_key).await;
assert!(!write_cache.file_cache.contains_key(&index_key));
}
#[tokio::test]
async fn test_read_metadata_from_write_cache() {
common_telemetry::init_default_ut_logging();
let mut env = TestEnv::new();
let data_home = env.data_home().display().to_string();
let mock_store = env.init_object_store_manager();
let local_dir = create_temp_dir("");
let local_path = local_dir.path().to_str().unwrap();
let local_store = new_fs_store(local_path);
let write_cache = env
.create_write_cache(local_store.clone(), ReadableSize::mb(10))
.await;
let cache_manager = Arc::new(
CacheManager::builder()
.write_cache(Some(write_cache.clone()))
.build(),
);
let metadata = Arc::new(sst_region_metadata());
let source = new_source(&[
new_batch_by_range(&["a", "d"], 0, 60),
new_batch_by_range(&["b", "f"], 0, 40),
new_batch_by_range(&["b", "h"], 100, 200),
]);
let write_request = SstWriteRequest {
op_type: OperationType::Flush,
metadata,
source,
storage: None,
max_sequence: None,
cache_manager: cache_manager.clone(),
index_options: IndexOptions::default(),
inverted_index_config: Default::default(),
fulltext_index_config: Default::default(),
bloom_filter_index_config: Default::default(),
};
let write_opts = WriteOptions {
row_group_size: 512,
..Default::default()
};
let upload_request = SstUploadRequest {
dest_path_provider: RegionFilePathFactory::new(data_home.clone()),
remote_store: mock_store.clone(),
};
let sst_info = write_cache
.write_and_upload_sst(write_request, upload_request, &write_opts)
.await
.unwrap()
.remove(0);
let write_parquet_metadata = sst_info.file_metadata.unwrap();
let handle = sst_file_handle_with_file_id(sst_info.file_id, 0, 1000);
let builder = ParquetReaderBuilder::new(data_home, handle.clone(), mock_store.clone())
.cache(CacheStrategy::EnableAll(cache_manager.clone()));
let reader = builder.build().await.unwrap();
assert_parquet_metadata_eq(write_parquet_metadata, reader.parquet_metadata());
}
}