From 3a91370a812bd09443fe542035da506797384281 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Sat, 20 Jul 2024 10:31:29 +0530 Subject: [PATCH 01/12] feat: Add hot-tier feature for distributed deployments - Env var P_HOT_TIER_DIR to store the files in the query server - PUT /logstream/{logstream}/hottier with below JSON body to set hottier for stream `{ "hotTierSize": "1GiB", "hotTierStartDate": "2024-07-16", "hotTierEndDate": "2024-07-19" }` - GET /logstream/{logstream}/hottier to fetch the current state of hottier for a stream Response JSON - `{ "hotTierSize": "1GiB", "hot_tier_used_size": "1019.41 MiB", "hot_tier_available_size": "4.59 MiB", "hotTierStartDate": "2024-07-16", "hotTierEndDate": "2024-07-19" }` Ingestion flow completed - - Query server periodically (every 1 minute) downloads the parquet files and corresponding manifest file from S3 for the range provided in the hot tier - deletes files if downloaded file size exceeds the hot tier size --- server/src/cli.rs | 15 + server/src/handlers/http/logstream.rs | 106 +++++- .../src/handlers/http/modal/query_server.rs | 6 +- server/src/handlers/http/modal/server.rs | 14 + server/src/hottier.rs | 318 ++++++++++++++++++ server/src/main.rs | 1 + server/src/metadata.rs | 19 ++ server/src/option.rs | 30 +- server/src/rbac/role.rs | 7 + server/src/storage.rs | 3 + server/src/storage/localfs.rs | 5 + server/src/storage/object_storage.rs | 1 + server/src/storage/s3.rs | 33 ++ server/src/validator.rs | 43 +++ 14 files changed, 592 insertions(+), 9 deletions(-) create mode 100644 server/src/hottier.rs diff --git a/server/src/cli.rs b/server/src/cli.rs index cec579e9a..3787fb721 100644 --- a/server/src/cli.rs +++ b/server/src/cli.rs @@ -101,6 +101,9 @@ pub struct Cli { /// CORS behaviour pub cors: bool, + + /// The local hot_tier path is used for optimising the query performance in the distributed systems + pub hot_tier_storage_path: Option, } impl Cli { @@ -134,6 +137,7 @@ impl Cli { pub const DEFAULT_PASSWORD: &'static str = "admin"; pub const FLIGHT_PORT: &'static str = "flight-port"; pub const CORS: &'static str = "cors"; + pub const HOT_TIER_PATH: &'static str = "hot-tier-path"; pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) @@ -395,6 +399,15 @@ impl Cli { "lz4", "zstd"]) .help("Parquet compression algorithm"), + ) + .arg( + Arg::new(Self::HOT_TIER_PATH) + .long(Self::HOT_TIER_PATH) + .env("P_HOT_TIER_DIR") + .value_name("DIR") + .value_parser(validation::canonicalize_path) + .help("Local path on this device to be used for hot tier data") + .next_line_help(true), ).group( ArgGroup::new("oidc") .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) @@ -532,6 +545,8 @@ impl FromArgMatches for Cli { _ => unreachable!(), }; + self.hot_tier_storage_path = m.get_one::(Self::HOT_TIER_PATH).cloned(); + Ok(()) } } diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index b13cd1679..7892ff2aa 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -27,6 +27,7 @@ use crate::handlers::{ CUSTOM_PARTITION_KEY, STATIC_SCHEMA_FLAG, TIME_PARTITION_KEY, TIME_PARTITION_LIMIT_KEY, UPDATE_STREAM_KEY, }; +use crate::hottier::{HotTierManager, StreamHotTier}; use crate::metadata::STREAM_INFO; use crate::metrics::{EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE_DATE, EVENTS_STORAGE_SIZE_DATE}; use crate::option::{Mode, CONFIG}; @@ -37,6 +38,7 @@ use crate::{ catalog::{self, remove_manifest_from_snapshot}, event, stats, }; + use crate::{metadata, validator}; use actix_web::http::StatusCode; use actix_web::{web, HttpRequest, Responder}; @@ -919,6 +921,91 @@ pub async fn get_stream_info(req: HttpRequest) -> Result, +) -> Result { + if CONFIG.parseable.mode != Mode::Query { + return Err(StreamError::Custom { + msg: "Hot tier can only be enabled in query mode".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + let stream_name: String = req.match_info().get("logstream").unwrap().parse().unwrap(); + if !metadata::STREAM_INFO.stream_exists(&stream_name) { + return Err(StreamError::StreamNotFound(stream_name)); + } + if CONFIG.parseable.hot_tier_storage_path.is_none() { + return Err(StreamError::HotTierNotEnabled(stream_name)); + } + + let body = body.into_inner(); + let hottier: StreamHotTier = match serde_json::from_value(body) { + Ok(hottier) => hottier, + Err(err) => return Err(StreamError::InvalidHotTierConfig(err)), + }; + + validator::hot_tier( + &hottier.hot_tier_start_date, + &hottier.hot_tier_end_date, + &hottier.hot_tier_size.to_string(), + )?; + + let storage = CONFIG.storage().get_object_store(); + let mut stream_metadata = storage.get_object_store_format(&stream_name).await?; + stream_metadata.hot_tier_enabled = Some(true); + storage + .put_stream_manifest(&stream_name, &stream_metadata) + .await?; + + STREAM_INFO.set_hot_tier(&stream_name, true)?; + if let Some(hot_tier_manager) = HotTierManager::global() { + let hottier = StreamHotTier { + hot_tier_start_date: hottier.hot_tier_start_date, + hot_tier_end_date: hottier.hot_tier_end_date, + hot_tier_size: hottier.hot_tier_size.clone(), + hot_tier_used_size: Some(0.to_string()), + hot_tier_available_size: Some(hottier.hot_tier_size), + }; + hot_tier_manager + .put_hot_tier(&stream_name, &hottier) + .await?; + } + + Ok(( + format!("hot tier set for log stream {stream_name}"), + StatusCode::OK, + )) +} + +pub async fn get_stream_hot_tier(req: HttpRequest) -> Result { + if CONFIG.parseable.mode != Mode::Query { + return Err(StreamError::Custom { + msg: "Hot tier can only be enabled in query mode".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + + let stream_name: String = req.match_info().get("logstream").unwrap().parse().unwrap(); + + if !metadata::STREAM_INFO.stream_exists(&stream_name) { + return Err(StreamError::StreamNotFound(stream_name)); + } + + if CONFIG.parseable.hot_tier_storage_path.is_none() { + return Err(StreamError::HotTierNotEnabled(stream_name)); + } + + if let Some(hot_tier_manager) = HotTierManager::global() { + let hot_tier = hot_tier_manager.get_hot_tier(&stream_name).await?; + Ok((web::Json(hot_tier), StatusCode::OK)) + } else { + Err(StreamError::Custom { + msg: format!("hot tier not initialised for stream {}", stream_name), + status: (StatusCode::BAD_REQUEST), + }) + } +} #[allow(unused)] fn classify_json_error(kind: serde_json::error::Category) -> StatusCode { match kind { @@ -935,9 +1022,12 @@ pub mod error { use http::StatusCode; use crate::{ + hottier::HotTierError, metadata::error::stream_info::MetadataError, storage::ObjectStorageError, - validator::error::{AlertValidationError, StreamNameValidationError}, + validator::error::{ + AlertValidationError, HotTierValidationError, StreamNameValidationError, + }, }; #[allow(unused)] @@ -997,6 +1087,16 @@ pub mod error { Network(#[from] reqwest::Error), #[error("Could not deserialize into JSON object, {0}")] SerdeError(#[from] serde_json::Error), + #[error( + "Hot tier is not enabled at the server config, cannot enable hot tier for stream {0}" + )] + HotTierNotEnabled(String), + #[error("failed to enable hottier due to err: {0}")] + InvalidHotTierConfig(serde_json::Error), + #[error("Hot tier validation failed due to {0}")] + HotTierValidation(#[from] HotTierValidationError), + #[error("Failed to update stream hot tier due to err: {0}")] + HotTierError(#[from] HotTierError), } impl actix_web::ResponseError for StreamError { @@ -1030,6 +1130,10 @@ pub mod error { StreamError::Network(err) => { err.status().unwrap_or(StatusCode::INTERNAL_SERVER_ERROR) } + StreamError::HotTierNotEnabled(_) => StatusCode::BAD_REQUEST, + StreamError::InvalidHotTierConfig(_) => StatusCode::BAD_REQUEST, + StreamError::HotTierValidation(_) => StatusCode::BAD_REQUEST, + StreamError::HotTierError(_) => StatusCode::INTERNAL_SERVER_ERROR, } } diff --git a/server/src/handlers/http/modal/query_server.rs b/server/src/handlers/http/modal/query_server.rs index c1323f8e2..166524c21 100644 --- a/server/src/handlers/http/modal/query_server.rs +++ b/server/src/handlers/http/modal/query_server.rs @@ -20,7 +20,7 @@ use crate::handlers::airplane; use crate::handlers::http::cluster::{self, init_cluster_metrics_schedular}; use crate::handlers::http::middleware::RouteExt; use crate::handlers::http::{base_path, cross_origin_config, API_BASE_PATH, API_VERSION}; - +use crate::hottier::HotTierManager; use crate::rbac::role::Action; use crate::sync; use crate::users::dashboards::DASHBOARDS; @@ -188,6 +188,10 @@ impl QueryServer { if matches!(init_cluster_metrics_schedular(), Ok(())) { log::info!("Cluster metrics scheduler started successfully"); } + + if let Some(hot_tier_manager) = HotTierManager::global() { + hot_tier_manager.download_from_s3()?; + }; let (localsync_handler, mut localsync_outbox, localsync_inbox) = sync::run_local_sync(); let (mut remote_sync_handler, mut remote_sync_outbox, mut remote_sync_inbox) = sync::object_store_sync(); diff --git a/server/src/handlers/http/modal/server.rs b/server/src/handlers/http/modal/server.rs index ceaf35810..1878edc47 100644 --- a/server/src/handlers/http/modal/server.rs +++ b/server/src/handlers/http/modal/server.rs @@ -343,6 +343,20 @@ impl Server { .to(logstream::get_cache_enabled) .authorize_for_stream(Action::GetCacheEnabled), ), + ) + .service( + web::resource("/hottier") + // PUT "/logstream/{logstream}/hottier" ==> Set hottier for given logstream + .route( + web::put() + .to(logstream::put_stream_hot_tier) + .authorize_for_stream(Action::PutHotTierEnabled), + ) + .route( + web::get() + .to(logstream::get_stream_hot_tier) + .authorize_for_stream(Action::GetHotTierEnabled), + ), ), ) } diff --git a/server/src/hottier.rs b/server/src/hottier.rs new file mode 100644 index 000000000..758bc0c4c --- /dev/null +++ b/server/src/hottier.rs @@ -0,0 +1,318 @@ +/* + * Parseable Server (C) 2022 - 2024 Parseable, Inc. + * + * This program is free software: you can redistribute it and/or modify + * it under the terms of the GNU Affero General Public License as + * published by the Free Software Foundation, either version 3 of the + * License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU Affero General Public License for more details. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + * + */ + +use std::{io, path::PathBuf}; + +use crate::{ + catalog::manifest::{File, Manifest}, + metadata::{error::stream_info::MetadataError, STREAM_INFO}, + option::{ + validation::{bytes_to_human_size, human_size_to_bytes}, + CONFIG, + }, + storage::ObjectStorageError, +}; +use chrono::NaiveDate; +use clokwerk::{AsyncScheduler, Interval}; +use futures_util::TryFutureExt; +use object_store::{local::LocalFileSystem, ObjectStore}; +use once_cell::sync::OnceCell; +use parquet::errors::ParquetError; +use relative_path::RelativePathBuf; +use std::time::Duration; +use tokio::fs::{self}; +use tokio::io::AsyncWriteExt; + +pub const STREAM_HOT_TIER_FILENAME: &str = ".hot_tier.json"; + +const HOT_TIER_SYNC_DURATION: Interval = clokwerk::Interval::Minutes(1); + +#[derive(Debug, serde::Deserialize, serde::Serialize)] +pub struct StreamHotTier { + #[serde(rename = "hotTierSize")] + pub hot_tier_size: String, + #[serde(skip_serializing_if = "Option::is_none")] + pub hot_tier_used_size: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub hot_tier_available_size: Option, + #[serde(rename = "hotTierStartDate")] + pub hot_tier_start_date: String, + #[serde(rename = "hotTierEndDate")] + pub hot_tier_end_date: String, +} + +pub struct HotTierManager { + filesystem: LocalFileSystem, + hot_tier_path: PathBuf, +} + +impl HotTierManager { + pub fn global() -> Option<&'static HotTierManager> { + static INSTANCE: OnceCell = OnceCell::new(); + + let hot_tier_path = CONFIG.parseable.hot_tier_storage_path.as_ref()?; + + Some(INSTANCE.get_or_init(|| { + let hot_tier_path = hot_tier_path.clone(); + std::fs::create_dir_all(&hot_tier_path).unwrap(); + HotTierManager { + filesystem: LocalFileSystem::new(), + hot_tier_path, + } + })) + } + + pub async fn get_hot_tier(&self, stream: &str) -> Result { + let path = hot_tier_file_path(&self.hot_tier_path, stream)?; + let res = self + .filesystem + .get(&path) + .and_then(|resp| resp.bytes()) + .await; + match res { + Ok(bytes) => serde_json::from_slice(&bytes).map_err(Into::into), + Err(err) => Err(err.into()), + } + } + + pub async fn put_hot_tier( + &self, + stream: &str, + hot_tier: &StreamHotTier, + ) -> Result<(), HotTierError> { + let path = hot_tier_file_path(&self.hot_tier_path, stream)?; + let bytes = serde_json::to_vec(hot_tier)?.into(); + self.filesystem.put(&path, bytes).await?; + Ok(()) + } + + pub fn download_from_s3<'a>(&'a self) -> Result<(), HotTierError> + where + 'a: 'static, + { + let mut scheduler = AsyncScheduler::new(); + scheduler.every(HOT_TIER_SYNC_DURATION).run(move || async { + if let Err(err) = self.sync_hot_tier().await { + log::error!("Error in hot tier scheduler: {:?}", err); + } + }); + + tokio::spawn(async move { + loop { + scheduler.run_pending().await; + tokio::time::sleep(Duration::from_secs(10)).await; + } + }); + Ok(()) + } + + async fn sync_hot_tier(&self) -> Result<(), HotTierError> { + let streams = STREAM_INFO.list_streams(); + for stream in streams { + if STREAM_INFO.get_hot_tier(&stream).unwrap_or(false) { + self.process_stream(stream).await?; + } + } + Ok(()) + } + + async fn process_stream(&self, stream: String) -> Result<(), HotTierError> { + let mut stream_hot_tier = self.get_hot_tier(&stream).await?; + let mut parquet_file_size = + human_size_to_bytes(&stream_hot_tier.hot_tier_used_size.clone().unwrap()).unwrap(); + let date_list = self.get_date_list( + &stream_hot_tier.hot_tier_start_date, + &stream_hot_tier.hot_tier_end_date, + )?; + + let object_store = CONFIG.storage().get_object_store(); + let s3_file_list = object_store.list_files(&stream).await?; + + for date in date_list { + self.process_date( + &stream, + &mut stream_hot_tier, + &s3_file_list, + date, + &mut parquet_file_size, + ) + .await?; + } + + self.put_hot_tier(&stream, &stream_hot_tier).await?; + Ok(()) + } + + fn get_date_list( + &self, + start_date: &str, + end_date: &str, + ) -> Result, HotTierError> { + let start_date = NaiveDate::parse_from_str(start_date, "%Y-%m-%d").unwrap(); + let end_date = NaiveDate::parse_from_str(end_date, "%Y-%m-%d").unwrap(); + let mut date_list = Vec::new(); + let mut current_date = start_date; + while current_date <= end_date { + date_list.push(current_date); + current_date += chrono::Duration::days(1); + } + Ok(date_list) + } + + async fn process_date( + &self, + stream: &str, + stream_hot_tier: &mut StreamHotTier, + s3_file_list: &[String], + date: NaiveDate, + parquet_file_size: &mut u64, + ) -> Result<(), HotTierError> { + let date_str = date.to_string(); + let manifest_files_to_download = s3_file_list + .iter() + .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) + .collect::>(); + + for file in manifest_files_to_download { + let path = self.hot_tier_path.join(file); + fs::create_dir_all(path.parent().unwrap()).await?; + let manifest_path: RelativePathBuf = RelativePathBuf::from(file); + let manifest_file = CONFIG + .storage() + .get_object_store() + .get_object(&manifest_path) + .await?; + let mut file = fs::File::create(path.clone()).await?; + file.write_all(&manifest_file).await?; + let manifest: Manifest = serde_json::from_slice(&manifest_file)?; + + for parquet_file in manifest.files { + self.process_parquet_file( + stream, + stream_hot_tier, + &parquet_file, + parquet_file_size, + ) + .await?; + } + } + Ok(()) + } + + async fn process_parquet_file( + &self, + stream: &str, + stream_hot_tier: &mut StreamHotTier, + parquet_file: &File, + parquet_file_size: &mut u64, + ) -> Result<(), HotTierError> { + let parquet_file_path = &parquet_file.file_path; + let parquet_path = self.hot_tier_path.join(parquet_file_path); + + if !parquet_path.exists() { + fs::create_dir_all(parquet_path.parent().unwrap()).await?; + let parquet_file_path = RelativePathBuf::from(parquet_file_path); + + if human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) + .unwrap() + <= parquet_file.file_size + { + self.delete_from_hot_tier( + stream, + &self.get_date_list( + &stream_hot_tier.hot_tier_start_date, + &stream_hot_tier.hot_tier_end_date, + )?[0], + ) + .await?; + fs::create_dir_all(parquet_path.parent().unwrap()).await?; + let mut file = fs::File::create(parquet_path.clone()).await?; + let parquet_data = CONFIG + .storage() + .get_object_store() + .get_object(&parquet_file_path) + .await?; + file.write_all(&parquet_data).await?; + *parquet_file_size += parquet_file.file_size; + stream_hot_tier.hot_tier_used_size = Some(bytes_to_human_size(*parquet_file_size)); + stream_hot_tier.hot_tier_available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) + .unwrap() + - parquet_file.file_size, + )); + } else { + let mut file = fs::File::create(parquet_path.clone()).await?; + let parquet_data = CONFIG + .storage() + .get_object_store() + .get_object(&parquet_file_path) + .await?; + file.write_all(&parquet_data).await?; + *parquet_file_size += parquet_file.file_size; + stream_hot_tier.hot_tier_used_size = Some(bytes_to_human_size(*parquet_file_size)); + stream_hot_tier.hot_tier_available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) + .unwrap() + - parquet_file.file_size, + )); + } + } + Ok(()) + } + + pub async fn delete_from_hot_tier( + &self, + stream: &str, + date: &NaiveDate, + ) -> Result<(), HotTierError> { + let path = self.hot_tier_path.join(format!("{}/date={}", stream, date)); + if path.exists() { + fs::remove_dir_all(path).await?; + } + + Ok(()) + } +} + +pub fn hot_tier_file_path( + root: impl AsRef, + stream: &str, +) -> Result { + let path = root.as_ref().join(stream).join(STREAM_HOT_TIER_FILENAME); + object_store::path::Path::from_absolute_path(path) +} + +#[derive(Debug, thiserror::Error)] +pub enum HotTierError { + #[error("{0}")] + Serde(#[from] serde_json::Error), + #[error("{0}")] + IOError(#[from] io::Error), + #[error("{0}")] + MoveError(#[from] fs_extra::error::Error), + #[error("{0}")] + ObjectStoreError(#[from] object_store::Error), + #[error("{0}")] + ObjectStorePathError(#[from] object_store::path::Error), + #[error("{0}")] + ObjectStorageError(#[from] ObjectStorageError), + #[error("{0}")] + ParquetError(#[from] ParquetError), + #[error("{0}")] + MetadataError(#[from] MetadataError), +} diff --git a/server/src/main.rs b/server/src/main.rs index e9d92abfc..fca2ca307 100644 --- a/server/src/main.rs +++ b/server/src/main.rs @@ -24,6 +24,7 @@ mod catalog; mod cli; mod event; mod handlers; +mod hottier; mod livetail; mod localcache; mod metadata; diff --git a/server/src/metadata.rs b/server/src/metadata.rs index 06eb995e7..b3f0869f3 100644 --- a/server/src/metadata.rs +++ b/server/src/metadata.rs @@ -61,6 +61,7 @@ pub struct LogStreamMetadata { pub time_partition_limit: Option, pub custom_partition: Option, pub static_schema_flag: Option, + pub hot_tier_enabled: Option, } // It is very unlikely that panic will occur when dealing with metadata. @@ -248,6 +249,22 @@ impl StreamInfo { Ok(()) } + pub fn set_hot_tier(&self, stream_name: &str, enable: bool) -> Result<(), MetadataError> { + let mut map = self.write().expect(LOCK_EXPECT); + let stream = map + .get_mut(stream_name) + .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string()))?; + stream.hot_tier_enabled = Some(enable); + Ok(()) + } + + pub fn get_hot_tier(&self, stream_name: &str) -> Result { + let map = self.read().expect(LOCK_EXPECT); + map.get(stream_name) + .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) + .map(|metadata| metadata.hot_tier_enabled.unwrap_or(false)) + } + #[allow(clippy::too_many_arguments)] pub fn add_stream( &self, @@ -330,6 +347,7 @@ impl StreamInfo { time_partition_limit: meta.time_partition_limit, custom_partition: meta.custom_partition, static_schema_flag: meta.static_schema_flag, + hot_tier_enabled: meta.hot_tier_enabled, }; let mut map = self.write().expect(LOCK_EXPECT); @@ -459,6 +477,7 @@ pub async fn load_stream_metadata_on_server_start( time_partition_limit, custom_partition, static_schema_flag: meta.static_schema_flag.clone(), + hot_tier_enabled: meta.hot_tier_enabled, }; let mut map = STREAM_INFO.write().expect(LOCK_EXPECT); diff --git a/server/src/option.rs b/server/src/option.rs index 06e693405..01b75a567 100644 --- a/server/src/option.rs +++ b/server/src/option.rs @@ -328,7 +328,7 @@ pub mod validation { url::Url::parse(s).map_err(|_| "Invalid URL provided".to_string()) } - fn human_size_to_bytes(s: &str) -> Result { + pub fn human_size_to_bytes(s: &str) -> Result { fn parse_and_map( s: &str, ) -> Result { @@ -343,15 +343,31 @@ pub mod validation { .or(parse_and_map::(s)) .map_err(|_| "Could not parse given size".to_string())?; - if size < MIN_CACHE_SIZE_BYTES { - return Err( - "Specified value of cache size is smaller than current minimum of 1GiB".to_string(), - ); - } - Ok(size) } + pub fn bytes_to_human_size(bytes: u64) -> String { + const KIB: u64 = 1024; + const MIB: u64 = KIB * 1024; + const GIB: u64 = MIB * 1024; + const TIB: u64 = GIB * 1024; + const PIB: u64 = TIB * 1024; + + if bytes < KIB { + format!("{} B", bytes) + } else if bytes < MIB { + format!("{:.2} KB", bytes as f64 / KIB as f64) + } else if bytes < GIB { + format!("{:.2} MiB", bytes as f64 / MIB as f64) + } else if bytes < TIB { + format!("{:.2} GiB", bytes as f64 / GIB as f64) + } else if bytes < PIB { + format!("{:.2} TiB", bytes as f64 / TIB as f64) + } else { + format!("{:.2} PiB", bytes as f64 / PIB as f64) + } + } + pub fn cache_size(s: &str) -> Result { let size = human_size_to_bytes(s)?; if size < MIN_CACHE_SIZE_BYTES { diff --git a/server/src/rbac/role.rs b/server/src/rbac/role.rs index accbd471a..0edc1bf7c 100644 --- a/server/src/rbac/role.rs +++ b/server/src/rbac/role.rs @@ -32,6 +32,8 @@ pub enum Action { PutRetention, GetCacheEnabled, PutCacheEnabled, + PutHotTierEnabled, + GetHotTierEnabled, PutAlert, GetAlert, PutUser, @@ -117,6 +119,8 @@ impl RoleBuilder { | Action::ListCluster | Action::ListClusterMetrics | Action::Deleteingestor + | Action::PutHotTierEnabled + | Action::GetHotTierEnabled | Action::ListDashboard | Action::GetDashboard | Action::CreateDashboard @@ -206,6 +210,8 @@ pub mod model { Action::PutRetention, Action::PutCacheEnabled, Action::GetCacheEnabled, + Action::PutHotTierEnabled, + Action::GetHotTierEnabled, Action::PutAlert, Action::GetAlert, Action::GetAbout, @@ -249,6 +255,7 @@ pub mod model { Action::GetAbout, Action::QueryLLM, Action::ListCluster, + Action::GetHotTierEnabled, ], stream: None, tag: None, diff --git a/server/src/storage.rs b/server/src/storage.rs index d28dcb00e..fe081b617 100644 --- a/server/src/storage.rs +++ b/server/src/storage.rs @@ -100,6 +100,8 @@ pub struct ObjectStoreFormat { pub custom_partition: Option, #[serde(skip_serializing_if = "Option::is_none")] pub static_schema_flag: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub hot_tier_enabled: Option, } #[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] @@ -167,6 +169,7 @@ impl Default for ObjectStoreFormat { time_partition_limit: None, custom_partition: None, static_schema_flag: None, + hot_tier_enabled: None, } } } diff --git a/server/src/storage/localfs.rs b/server/src/storage/localfs.rs index 78b02e8d5..d914e2693 100644 --- a/server/src/storage/localfs.rs +++ b/server/src/storage/localfs.rs @@ -411,6 +411,11 @@ impl ObjectStorage for LocalFS { Ok(dates.into_iter().flatten().collect()) } + async fn list_files(&self, _stream_name: &str) -> Result, ObjectStorageError> { + //unimplemented + Ok(vec![]) + } + async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError> { let op = CopyOptions { overwrite: true, diff --git a/server/src/storage/object_storage.rs b/server/src/storage/object_storage.rs index 433327778..e8e045f23 100644 --- a/server/src/storage/object_storage.rs +++ b/server/src/storage/object_storage.rs @@ -87,6 +87,7 @@ pub trait ObjectStorage: Sync + 'static { async fn get_all_saved_filters(&self) -> Result, ObjectStorageError>; async fn get_all_dashboards(&self) -> Result, ObjectStorageError>; async fn list_dates(&self, stream_name: &str) -> Result, ObjectStorageError>; + async fn list_files(&self, stream_name: &str) -> Result, ObjectStorageError>; async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError>; async fn delete_object(&self, path: &RelativePath) -> Result<(), ObjectStorageError>; async fn get_ingestor_meta_file_paths( diff --git a/server/src/storage/s3.rs b/server/src/storage/s3.rs index d94b937d3..318172899 100644 --- a/server/src/storage/s3.rs +++ b/server/src/storage/s3.rs @@ -343,6 +343,33 @@ impl S3 { Ok(dates) } + async fn _list_files(&self, stream: &str) -> Result, ObjectStorageError> { + let mut result_file_list = vec![]; + let resp = self + .client + .list_with_delimiter(Some(&(stream.into()))) + .await?; + + let dates = resp + .common_prefixes + .iter() + .flat_map(|path| path.parts()) + .filter(|name| name.as_ref() != stream && name.as_ref() != STREAM_ROOT_DIRECTORY) + .map(|name| name.as_ref().to_string()) + .collect::>(); + for date in dates.clone() { + let hour_path = object_store::path::Path::from(format!("{}/{}", stream, date)); + let resp = self.client.list_with_delimiter(Some(&hour_path)).await?; + let manifest = resp + .objects + .iter() + .map(|name| name.location.to_string()) + .collect::(); + result_file_list.push(manifest); + } + + Ok(result_file_list) + } async fn _upload_file(&self, key: &str, path: &StdPath) -> Result<(), ObjectStorageError> { let instant = Instant::now(); @@ -604,6 +631,12 @@ impl ObjectStorage for S3 { Ok(streams) } + async fn list_files(&self, stream_name: &str) -> Result, ObjectStorageError> { + let files = self._list_files(stream_name).await?; + + Ok(files) + } + async fn upload_file(&self, key: &str, path: &StdPath) -> Result<(), ObjectStorageError> { self._upload_file(key, path).await?; diff --git a/server/src/validator.rs b/server/src/validator.rs index a1648ff22..efcb3d1e9 100644 --- a/server/src/validator.rs +++ b/server/src/validator.rs @@ -16,11 +16,15 @@ * */ +use chrono::NaiveDate; +use error::HotTierValidationError; + use self::error::{AlertValidationError, StreamNameValidationError, UsernameValidationError}; use crate::alerts::rule::base::{NumericRule, StringRule}; use crate::alerts::rule::{ColumnRule, ConsecutiveNumericRule, ConsecutiveStringRule}; use crate::alerts::{Alerts, Rule}; use crate::handlers::http::cluster::INTERNAL_STREAM_NAME; +use crate::option::validation::{cache_size, human_size_to_bytes}; // Add more sql keywords here in lower case const DENIED_NAMES: &[&str] = &[ @@ -144,6 +148,29 @@ pub fn user_name(username: &str) -> Result<(), UsernameValidationError> { Ok(()) } +pub fn hot_tier( + start_date: &str, + end_date: &str, + size: &str, +) -> Result<(), HotTierValidationError> { + if human_size_to_bytes(size).is_err() { + return Err(HotTierValidationError::Size); + } + cache_size(size).map_err(|_| HotTierValidationError::Size)?; + + let start_date: NaiveDate = start_date + .parse() + .map_err(|_| HotTierValidationError::StartDate)?; + let end_date: NaiveDate = end_date + .parse() + .map_err(|_| HotTierValidationError::EndDate)?; + + if start_date > end_date { + return Err(HotTierValidationError::DateRange); + } + + Ok(()) +} pub mod error { #[derive(Debug, thiserror::Error)] @@ -191,4 +218,20 @@ pub mod error { )] SpecialChar, } + + #[derive(Debug, thiserror::Error)] + pub enum HotTierValidationError { + #[error("Invalid size given for hot tier, please provide size in human readable format, e.g 1GiB, 2GiB")] + Size, + #[error( + "Invalid start date given for hot tier, please provide the date in yyyy-mm-dd format" + )] + StartDate, + #[error( + "Invalid end date given for hot tier, please provide the date in yyyy-mm-dd format" + )] + EndDate, + #[error("End date should be greater than start date")] + DateRange, + } } From 5d007bd8b7c2e1519375b4ed91d6d9523fda4369 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Wed, 24 Jul 2024 14:40:23 +0530 Subject: [PATCH 02/12] updated ingestion flow - 1. updated request JSON body for PUT /logstream/{logstream}/hottier ` { "size": "1GiB", "start_date": "1days", "end_date": "now" } ` Validations in place to set hottier for a stream - 1. Hot tier can be enabled in distributed mode by Querier node only 2. Stream should exist 3. Hot tier cannot be enabled for streams having time partitions 4. Minimum size of Hot tier is set to 1GiB 5. end_date can be set to now for current date 6. if end_date is not given as now, start and end date should be in format yyyy-mm-dd 7. You can set hot tier for a minimum of 2 days duration i.e. end_date > start_date 8. Size given in request will be matched against the total size of files to be downloaded, if size of hot tier is found to be less than the total size of files, validation fails Schedular to set to run every 1 min to verify and download new files from S3 If total size of hot tier is exhausted, oldest date entry in the hot tier (start date) will be deleted and hot tier start date will be updated to start date + 1 If hot tier is updated for an existing hot tier, all downloaded files will be deleted and files in the range of dates given in PUT request will be downloaded fresh from S3 The used size and available size in hot tier gets updated as and when files get downloaded/deleted from hot tier The same can be fetched using GET /logstream/{logstream}/hottier The response JSON will be like ` { "size": "1GiB", "used_size": "540MiB", "available_size": "460MiB", "start_date": "1days", "end_date": "now" } ` Added Query flow - 1. Server gets the stream.json and related manifest.json files from storage based on the query time range provided 2. then gets the list of parquet file path from manifest.json 3. it then checks for a list of parquet files available in hot tier 4. datafusion serves the results from hot tier for files available in hot tier 5. then serves the remaining files (not available in hot tier) from S3 --- server/src/handlers/http/logstream.rs | 32 +- server/src/hottier.rs | 327 ++++++++++++++++----- server/src/metadata.rs | 7 - server/src/query/stream_schema_provider.rs | 49 ++- server/src/utils.rs | 24 +- server/src/validator.rs | 46 ++- 6 files changed, 384 insertions(+), 101 deletions(-) diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 7892ff2aa..4a2225fed 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -939,6 +939,17 @@ pub async fn put_stream_hot_tier( return Err(StreamError::HotTierNotEnabled(stream_name)); } + if STREAM_INFO + .get_time_partition(&stream_name) + .unwrap() + .is_some() + { + return Err(StreamError::Custom { + msg: "Hot tier can not be enabled for stream with time partition".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + let body = body.into_inner(); let hottier: StreamHotTier = match serde_json::from_value(body) { Ok(hottier) => hottier, @@ -946,9 +957,9 @@ pub async fn put_stream_hot_tier( }; validator::hot_tier( - &hottier.hot_tier_start_date, - &hottier.hot_tier_end_date, - &hottier.hot_tier_size.to_string(), + &hottier.start_date, + &hottier.end_date, + &hottier.size.to_string(), )?; let storage = CONFIG.storage().get_object_store(); @@ -961,12 +972,15 @@ pub async fn put_stream_hot_tier( STREAM_INFO.set_hot_tier(&stream_name, true)?; if let Some(hot_tier_manager) = HotTierManager::global() { let hottier = StreamHotTier { - hot_tier_start_date: hottier.hot_tier_start_date, - hot_tier_end_date: hottier.hot_tier_end_date, - hot_tier_size: hottier.hot_tier_size.clone(), - hot_tier_used_size: Some(0.to_string()), - hot_tier_available_size: Some(hottier.hot_tier_size), + start_date: hottier.start_date, + end_date: hottier.end_date, + size: hottier.size.clone(), + used_size: Some("0GiB".to_string()), + available_size: Some(hottier.size), }; + + hot_tier_manager.validate(&stream_name, &hottier).await?; + hot_tier_manager .put_hot_tier(&stream_name, &hottier) .await?; @@ -1095,7 +1109,7 @@ pub mod error { InvalidHotTierConfig(serde_json::Error), #[error("Hot tier validation failed due to {0}")] HotTierValidation(#[from] HotTierValidationError), - #[error("Failed to update stream hot tier due to err: {0}")] + #[error("{0}")] HotTierError(#[from] HotTierError), } diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 758bc0c4c..b83b38e68 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -16,7 +16,7 @@ * */ -use std::{io, path::PathBuf}; +use std::{io, path::PathBuf, sync::Arc}; use crate::{ catalog::manifest::{File, Manifest}, @@ -25,18 +25,22 @@ use crate::{ validation::{bytes_to_human_size, human_size_to_bytes}, CONFIG, }, - storage::ObjectStorageError, + storage::{ObjectStorage, ObjectStorageError}, + utils::get_dir_size, + validator::{error::HotTierValidationError, parse_human_date}, }; use chrono::NaiveDate; -use clokwerk::{AsyncScheduler, Interval}; +use clokwerk::{AsyncScheduler, Interval, Job}; +use futures::TryStreamExt; use futures_util::TryFutureExt; use object_store::{local::LocalFileSystem, ObjectStore}; use once_cell::sync::OnceCell; use parquet::errors::ParquetError; use relative_path::RelativePathBuf; use std::time::Duration; -use tokio::fs::{self}; +use tokio::fs::{self, DirEntry}; use tokio::io::AsyncWriteExt; +use tokio_stream::wrappers::ReadDirStream; pub const STREAM_HOT_TIER_FILENAME: &str = ".hot_tier.json"; @@ -44,16 +48,16 @@ const HOT_TIER_SYNC_DURATION: Interval = clokwerk::Interval::Minutes(1); #[derive(Debug, serde::Deserialize, serde::Serialize)] pub struct StreamHotTier { - #[serde(rename = "hotTierSize")] - pub hot_tier_size: String, + #[serde(rename = "size")] + pub size: String, #[serde(skip_serializing_if = "Option::is_none")] - pub hot_tier_used_size: Option, + pub used_size: Option, #[serde(skip_serializing_if = "Option::is_none")] - pub hot_tier_available_size: Option, - #[serde(rename = "hotTierStartDate")] - pub hot_tier_start_date: String, - #[serde(rename = "hotTierEndDate")] - pub hot_tier_end_date: String, + pub available_size: Option, + #[serde(rename = "start_date")] + pub start_date: String, + #[serde(rename = "end_date")] + pub end_date: String, } pub struct HotTierManager { @@ -77,7 +81,65 @@ impl HotTierManager { })) } + pub async fn validate( + &self, + stream: &str, + stream_hot_tier: &StreamHotTier, + ) -> Result<(), HotTierError> { + let date_list = + self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let object_store = CONFIG.storage().get_object_store(); + let s3_file_list = object_store.list_files(stream).await?; + let mut manifest_list = Vec::new(); + let mut total_size_to_download = 0; + for date in date_list { + let date_str = date.to_string(); + let manifest_files_to_download = s3_file_list + .iter() + .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) + .collect::>(); + + for file in manifest_files_to_download { + let path = self.hot_tier_path.join(file); + fs::create_dir_all(path.parent().unwrap()).await?; + let manifest_path: RelativePathBuf = RelativePathBuf::from(file); + let manifest_file = object_store.get_object(&manifest_path).await?; + + let manifest: Manifest = serde_json::from_slice(&manifest_file)?; + manifest_list.push(manifest.clone()); + } + } + for manifest in &manifest_list { + total_size_to_download += manifest + .files + .iter() + .map(|file| file.file_size) + .sum::(); + } + if human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( + format!( + "Total size required to download the files - {}, size provided for the hot tier - {}, not enough space in hot tier", + bytes_to_human_size(total_size_to_download), + &stream_hot_tier.size + ), + ))); + } + if let Ok(mut existing_hot_tier) = self.get_hot_tier(stream).await { + let available_date_list = self.get_hot_tier_date_list(stream).await?; + self.delete_from_hot_tier(&mut existing_hot_tier, stream, &available_date_list, true) + .await?; + } + + Ok(()) + } + pub async fn get_hot_tier(&self, stream: &str) -> Result { + if !self.check_stream_hot_tier_exists(stream) { + return Err(HotTierError::HotTierValidationError( + HotTierValidationError::NotFound(stream.to_owned()), + )); + } let path = hot_tier_file_path(&self.hot_tier_path, stream)?; let res = self .filesystem @@ -106,11 +168,14 @@ impl HotTierManager { 'a: 'static, { let mut scheduler = AsyncScheduler::new(); - scheduler.every(HOT_TIER_SYNC_DURATION).run(move || async { - if let Err(err) = self.sync_hot_tier().await { - log::error!("Error in hot tier scheduler: {:?}", err); - } - }); + scheduler + .every(HOT_TIER_SYNC_DURATION) + .plus(Interval::Seconds(5)) + .run(move || async { + if let Err(err) = self.sync_hot_tier().await { + log::error!("Error in hot tier scheduler: {:?}", err); + } + }); tokio::spawn(async move { loop { @@ -124,7 +189,7 @@ impl HotTierManager { async fn sync_hot_tier(&self) -> Result<(), HotTierError> { let streams = STREAM_INFO.list_streams(); for stream in streams { - if STREAM_INFO.get_hot_tier(&stream).unwrap_or(false) { + if self.check_stream_hot_tier_exists(&stream) { self.process_stream(stream).await?; } } @@ -134,11 +199,19 @@ impl HotTierManager { async fn process_stream(&self, stream: String) -> Result<(), HotTierError> { let mut stream_hot_tier = self.get_hot_tier(&stream).await?; let mut parquet_file_size = - human_size_to_bytes(&stream_hot_tier.hot_tier_used_size.clone().unwrap()).unwrap(); - let date_list = self.get_date_list( - &stream_hot_tier.hot_tier_start_date, - &stream_hot_tier.hot_tier_end_date, - )?; + human_size_to_bytes(stream_hot_tier.used_size.as_ref().unwrap()).unwrap(); + let date_list = + self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let available_date_list = self.get_hot_tier_date_list(&stream).await?; + let dates_to_delete: Vec = available_date_list + .into_iter() + .filter(|available_date| !date_list.contains(available_date)) + .collect(); + + if !dates_to_delete.is_empty() { + self.delete_from_hot_tier(&mut stream_hot_tier, &stream, &dates_to_delete, false) + .await?; + } let object_store = CONFIG.storage().get_object_store(); let s3_file_list = object_store.list_files(&stream).await?; @@ -150,11 +223,12 @@ impl HotTierManager { &s3_file_list, date, &mut parquet_file_size, + object_store.clone(), ) .await?; + self.put_hot_tier(&stream, &stream_hot_tier).await?; } - self.put_hot_tier(&stream, &stream_hot_tier).await?; Ok(()) } @@ -163,14 +237,15 @@ impl HotTierManager { start_date: &str, end_date: &str, ) -> Result, HotTierError> { - let start_date = NaiveDate::parse_from_str(start_date, "%Y-%m-%d").unwrap(); - let end_date = NaiveDate::parse_from_str(end_date, "%Y-%m-%d").unwrap(); + let (start_date, end_date) = parse_human_date(start_date, end_date)?; let mut date_list = Vec::new(); let mut current_date = start_date; + while current_date <= end_date { date_list.push(current_date); current_date += chrono::Duration::days(1); } + Ok(date_list) } @@ -181,26 +256,27 @@ impl HotTierManager { s3_file_list: &[String], date: NaiveDate, parquet_file_size: &mut u64, + object_store: Arc, ) -> Result<(), HotTierError> { let date_str = date.to_string(); let manifest_files_to_download = s3_file_list .iter() .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) .collect::>(); - + let mut manifest_list = Vec::new(); for file in manifest_files_to_download { let path = self.hot_tier_path.join(file); fs::create_dir_all(path.parent().unwrap()).await?; let manifest_path: RelativePathBuf = RelativePathBuf::from(file); - let manifest_file = CONFIG - .storage() - .get_object_store() - .get_object(&manifest_path) - .await?; + let manifest_file = object_store.get_object(&manifest_path).await?; + let mut file = fs::File::create(path.clone()).await?; file.write_all(&manifest_file).await?; let manifest: Manifest = serde_json::from_slice(&manifest_file)?; + manifest_list.push(manifest.clone()); + } + for manifest in manifest_list { for parquet_file in manifest.files { self.process_parquet_file( stream, @@ -228,65 +304,164 @@ impl HotTierManager { fs::create_dir_all(parquet_path.parent().unwrap()).await?; let parquet_file_path = RelativePathBuf::from(parquet_file_path); - if human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) - .unwrap() + if human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() <= parquet_file.file_size { - self.delete_from_hot_tier( - stream, - &self.get_date_list( - &stream_hot_tier.hot_tier_start_date, - &stream_hot_tier.hot_tier_end_date, - )?[0], - ) - .await?; - fs::create_dir_all(parquet_path.parent().unwrap()).await?; - let mut file = fs::File::create(parquet_path.clone()).await?; - let parquet_data = CONFIG - .storage() - .get_object_store() - .get_object(&parquet_file_path) + let date_list = + self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let date_to_delete = vec![*date_list.first().unwrap()]; + self.delete_from_hot_tier(stream_hot_tier, stream, &date_to_delete, false) .await?; - file.write_all(&parquet_data).await?; - *parquet_file_size += parquet_file.file_size; - stream_hot_tier.hot_tier_used_size = Some(bytes_to_human_size(*parquet_file_size)); - stream_hot_tier.hot_tier_available_size = Some(bytes_to_human_size( - human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) - .unwrap() - - parquet_file.file_size, - )); - } else { - let mut file = fs::File::create(parquet_path.clone()).await?; - let parquet_data = CONFIG - .storage() - .get_object_store() - .get_object(&parquet_file_path) - .await?; - file.write_all(&parquet_data).await?; - *parquet_file_size += parquet_file.file_size; - stream_hot_tier.hot_tier_used_size = Some(bytes_to_human_size(*parquet_file_size)); - stream_hot_tier.hot_tier_available_size = Some(bytes_to_human_size( - human_size_to_bytes(&stream_hot_tier.hot_tier_available_size.clone().unwrap()) - .unwrap() - - parquet_file.file_size, - )); + self.update_hot_tier(stream_hot_tier).await?; + *parquet_file_size = + human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap(); } + + fs::create_dir_all(parquet_path.parent().unwrap()).await?; + let mut file = fs::File::create(parquet_path.clone()).await?; + let parquet_data = CONFIG + .storage() + .get_object_store() + .get_object(&parquet_file_path) + .await?; + file.write_all(&parquet_data).await?; + *parquet_file_size += parquet_file.file_size; + stream_hot_tier.used_size = Some(bytes_to_human_size(*parquet_file_size)); + stream_hot_tier.available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() + - parquet_file.file_size, + )); } Ok(()) } pub async fn delete_from_hot_tier( &self, + stream_hot_tier: &mut StreamHotTier, stream: &str, - date: &NaiveDate, + dates: &[NaiveDate], + validate: bool, ) -> Result<(), HotTierError> { - let path = self.hot_tier_path.join(format!("{}/date={}", stream, date)); + for date in dates.iter() { + let path = self.hot_tier_path.join(format!("{}/date={}", stream, date)); + if path.exists() { + if !validate { + let size = get_dir_size(path.clone())?; + stream_hot_tier.used_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap() + - size, + )); + stream_hot_tier.available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()) + .unwrap() + + size, + )); + } + fs::remove_dir_all(path.clone()).await?; + } + } + + Ok(()) + } + + pub async fn get_hot_tier_date_list( + &self, + stream: &str, + ) -> Result, HotTierError> { + let mut date_list = Vec::new(); + let path = self.hot_tier_path.join(stream); if path.exists() { - fs::remove_dir_all(path).await?; + let directories = ReadDirStream::new(fs::read_dir(&path).await?); + let dates: Vec = directories.try_collect().await?; + for date in dates { + if !date.path().is_dir() { + continue; + } + let date = date.file_name().into_string().unwrap(); + date_list.push( + NaiveDate::parse_from_str(date.trim_start_matches("date="), "%Y-%m-%d") + .unwrap(), + ); + } } + Ok(date_list) + } + pub async fn update_hot_tier( + &self, + stream_hot_tier: &mut StreamHotTier, + ) -> Result<(), HotTierError> { + let start_date = &stream_hot_tier.start_date; + let end_date = &stream_hot_tier.end_date; + let mut date_list = self.get_date_list(start_date, end_date)?; + + date_list.retain(|date: &NaiveDate| *date.to_string() != *start_date); + stream_hot_tier.start_date = date_list.first().unwrap().to_string(); Ok(()) } + + pub async fn get_hot_tier_manifests( + &self, + stream: &str, + manifest_files: Vec, + ) -> Result<(Vec, Vec), HotTierError> { + let hot_tier_files = self.get_hot_tier_parquet_files(stream).await?; + let remaining_files: Vec = manifest_files + .into_iter() + .filter(|manifest_file| { + hot_tier_files + .iter() + .all(|file| !file.file_path.eq(&manifest_file.file_path)) + }) + .collect(); + Ok((hot_tier_files, remaining_files)) + } + + pub async fn get_hot_tier_parquet_files( + &self, + stream: &str, + ) -> Result, HotTierError> { + let mut hot_tier_parquet_files: Vec = Vec::new(); + let stream_hot_tier = self.get_hot_tier(stream).await?; + let date_list = + self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + for date in date_list { + let date_str = date.to_string(); + let path = &self + .hot_tier_path + .join(stream) + .join(format!("date={}", date_str)); + if !path.exists() { + continue; + } + + let date_dirs = ReadDirStream::new(fs::read_dir(&path).await?); + let manifest_files: Vec = date_dirs.try_collect().await?; + for manifest in manifest_files { + if !manifest + .file_name() + .to_string_lossy() + .ends_with(".manifest.json") + { + continue; + } + let file = fs::read(manifest.path()).await?; + let manifest: Manifest = serde_json::from_slice(&file)?; + + for parquet_file in manifest.files { + hot_tier_parquet_files.push(parquet_file.clone()); + } + } + } + Ok(hot_tier_parquet_files) + } + pub fn check_stream_hot_tier_exists(&self, stream: &str) -> bool { + let path = self + .hot_tier_path + .join(stream) + .join(STREAM_HOT_TIER_FILENAME); + path.exists() + } } pub fn hot_tier_file_path( @@ -315,4 +490,8 @@ pub enum HotTierError { ParquetError(#[from] ParquetError), #[error("{0}")] MetadataError(#[from] MetadataError), + #[error("{0}")] + HotTierValidationError(#[from] HotTierValidationError), + #[error("{0}")] + Anyhow(#[from] anyhow::Error), } diff --git a/server/src/metadata.rs b/server/src/metadata.rs index b3f0869f3..40b47720a 100644 --- a/server/src/metadata.rs +++ b/server/src/metadata.rs @@ -258,13 +258,6 @@ impl StreamInfo { Ok(()) } - pub fn get_hot_tier(&self, stream_name: &str) -> Result { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.hot_tier_enabled.unwrap_or(false)) - } - #[allow(clippy::too_many_arguments)] pub fn add_stream( &self, diff --git a/server/src/query/stream_schema_provider.rs b/server/src/query/stream_schema_provider.rs index 6f9eb5608..d9e94ece7 100644 --- a/server/src/query/stream_schema_provider.rs +++ b/server/src/query/stream_schema_provider.rs @@ -16,6 +16,7 @@ * */ +use crate::hottier::HotTierManager; use crate::Mode; use crate::{ catalog::snapshot::{self, Snapshot}, @@ -293,6 +294,7 @@ impl TableProvider for StandardTableProvider { ) -> Result, DataFusionError> { let mut memory_exec = None; let mut cache_exec = None; + let mut hot_tier_exec = None; let object_store = state .runtime_env() .object_store_registry @@ -416,10 +418,53 @@ impl TableProvider for StandardTableProvider { cache_exec = Some(plan) } + // Hot tier data fetch + if let Some(hot_tier_manager) = HotTierManager::global() { + if hot_tier_manager.check_stream_hot_tier_exists(&self.stream) { + let (hot_tier_files, remainder) = hot_tier_manager + .get_hot_tier_manifests(&self.stream, manifest_files) + .await + .map_err(|err| DataFusionError::External(Box::new(err)))?; + // Assign remaining entries back to manifest list + // This is to be used for remote query + manifest_files = remainder; + + let hot_tier_files = hot_tier_files + .into_iter() + .map(|mut file| { + let path = CONFIG + .parseable + .hot_tier_storage_path + .as_ref() + .unwrap() + .join(&file.file_path); + file.file_path = path.to_str().unwrap().to_string(); + file + }) + .collect(); + + let (partitioned_files, statistics) = + partitioned_files(hot_tier_files, &self.schema, 1); + let plan = create_parquet_physical_plan( + ObjectStoreUrl::parse("file:///").unwrap(), + partitioned_files, + statistics, + self.schema.clone(), + projection, + filters, + limit, + state, + time_partition.clone(), + ) + .await?; + + hot_tier_exec = Some(plan) + } + } if manifest_files.is_empty() { QUERY_CACHE_HIT.with_label_values(&[&self.stream]).inc(); return final_plan( - vec![memory_exec, cache_exec], + vec![memory_exec, cache_exec, hot_tier_exec], projection, self.schema.clone(), ); @@ -440,7 +485,7 @@ impl TableProvider for StandardTableProvider { .await?; Ok(final_plan( - vec![memory_exec, cache_exec, Some(remote_exec)], + vec![memory_exec, cache_exec, hot_tier_exec, Some(remote_exec)], projection, self.schema.clone(), )?) diff --git a/server/src/utils.rs b/server/src/utils.rs index b8435b25e..8565ada04 100644 --- a/server/src/utils.rs +++ b/server/src/utils.rs @@ -27,7 +27,8 @@ use chrono::{DateTime, NaiveDate, Timelike, Utc}; use itertools::Itertools; use sha2::{Digest, Sha256}; use std::collections::HashMap; -use std::env; +use std::path::PathBuf; +use std::{env, fs}; use url::Url; #[allow(dead_code)] @@ -305,6 +306,27 @@ pub fn get_ingestor_id() -> String { result } +pub fn get_dir_size(path: PathBuf) -> Result { + fn dir_size_recursive(path: &PathBuf) -> Result { + let mut size = 0; + + for entry in fs::read_dir(path)? { + let entry = entry?; + let metadata = entry.metadata()?; + + if metadata.is_dir() { + size += dir_size_recursive(&entry.path())?; + } else { + size += metadata.len(); + } + } + + Ok(size) + } + + dir_size_recursive(&path) +} + #[cfg(test)] mod tests { use chrono::DateTime; diff --git a/server/src/validator.rs b/server/src/validator.rs index efcb3d1e9..f163cb1e1 100644 --- a/server/src/validator.rs +++ b/server/src/validator.rs @@ -16,7 +16,7 @@ * */ -use chrono::NaiveDate; +use chrono::{NaiveDate, Utc}; use error::HotTierValidationError; use self::error::{AlertValidationError, StreamNameValidationError, UsernameValidationError}; @@ -158,19 +158,39 @@ pub fn hot_tier( } cache_size(size).map_err(|_| HotTierValidationError::Size)?; - let start_date: NaiveDate = start_date - .parse() - .map_err(|_| HotTierValidationError::StartDate)?; - let end_date: NaiveDate = end_date - .parse() - .map_err(|_| HotTierValidationError::EndDate)?; + let (start_date, end_date) = parse_human_date(start_date, end_date)?; - if start_date > end_date { + if start_date >= end_date { return Err(HotTierValidationError::DateRange); } Ok(()) } + +pub fn parse_human_date( + start_time: &str, + end_time: &str, +) -> Result<(NaiveDate, NaiveDate), HotTierValidationError> { + let start: NaiveDate; + let end: NaiveDate; + + if end_time == "now" { + end = Utc::now().naive_utc().date(); + start = if let Ok(parsed_date) = NaiveDate::parse_from_str(start_time, "%Y-%m-%d") { + parsed_date + } else { + end - chrono::Duration::from_std(humantime::parse_duration(start_time)?) + .map_err(|_| HotTierValidationError::StartDate)? + }; + } else { + start = NaiveDate::parse_from_str(start_time, "%Y-%m-%d") + .map_err(|_| HotTierValidationError::StartDate)?; + end = NaiveDate::parse_from_str(end_time, "%Y-%m-%d") + .map_err(|_| HotTierValidationError::EndDate)?; + }; + + Ok((start, end)) +} pub mod error { #[derive(Debug, thiserror::Error)] @@ -231,7 +251,17 @@ pub mod error { "Invalid end date given for hot tier, please provide the date in yyyy-mm-dd format" )] EndDate, + #[error("End date should be greater than start date")] DateRange, + + #[error("While generating times for 'now' failed to parse duration")] + NotValidDuration(#[from] humantime::DurationError), + + #[error("Parsed duration out of range")] + OutOfRange(#[from] chrono::OutOfRangeError), + + #[error("Hot tier not found for stream {0}")] + NotFound(String), } } From 6df94cd67d01e0788f7e80d6bd18ead7133eaad6 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Wed, 24 Jul 2024 16:20:11 +0530 Subject: [PATCH 03/12] hot tier for multiple ingestors each ingestor writes its own manifest file to storage query server should download all manifest files to hot tier and download all corresponding parquet files from storage fix in PUT /hottier complete all validation first then create hot tier directory then update in stream json and in memory --- server/src/handlers/http/logstream.rs | 13 ++++++------- server/src/storage/s3.rs | 6 +++--- 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 4a2225fed..34e8a3e3b 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -962,13 +962,6 @@ pub async fn put_stream_hot_tier( &hottier.size.to_string(), )?; - let storage = CONFIG.storage().get_object_store(); - let mut stream_metadata = storage.get_object_store_format(&stream_name).await?; - stream_metadata.hot_tier_enabled = Some(true); - storage - .put_stream_manifest(&stream_name, &stream_metadata) - .await?; - STREAM_INFO.set_hot_tier(&stream_name, true)?; if let Some(hot_tier_manager) = HotTierManager::global() { let hottier = StreamHotTier { @@ -984,6 +977,12 @@ pub async fn put_stream_hot_tier( hot_tier_manager .put_hot_tier(&stream_name, &hottier) .await?; + let storage = CONFIG.storage().get_object_store(); + let mut stream_metadata = storage.get_object_store_format(&stream_name).await?; + stream_metadata.hot_tier_enabled = Some(true); + storage + .put_stream_manifest(&stream_name, &stream_metadata) + .await?; } Ok(( diff --git a/server/src/storage/s3.rs b/server/src/storage/s3.rs index 318172899..d545fa354 100644 --- a/server/src/storage/s3.rs +++ b/server/src/storage/s3.rs @@ -360,12 +360,12 @@ impl S3 { for date in dates.clone() { let hour_path = object_store::path::Path::from(format!("{}/{}", stream, date)); let resp = self.client.list_with_delimiter(Some(&hour_path)).await?; - let manifest = resp + let manifest: Vec = resp .objects .iter() .map(|name| name.location.to_string()) - .collect::(); - result_file_list.push(manifest); + .collect(); + result_file_list.extend(manifest); } Ok(result_file_list) From b4b2e5fc8a3ed56e4ff704beb9e4559a78e5e58d Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Wed, 24 Jul 2024 23:56:36 +0530 Subject: [PATCH 04/12] enhancement to the hot-tier implementation below checks are added - if hot tier range does not have today's date and the manifest and respective parquet files are already downloaded no need to download the manifest again if stream is initialised today and S3 contains only today's data and the hot tier size is exhausted, today's data cannot be deleted log error and skip the download --- server/src/hottier.rs | 37 +++++++++++++++++++++++++++++++------ 1 file changed, 31 insertions(+), 6 deletions(-) diff --git a/server/src/hottier.rs b/server/src/hottier.rs index b83b38e68..b422a3f68 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -29,7 +29,7 @@ use crate::{ utils::get_dir_size, validator::{error::HotTierValidationError, parse_human_date}, }; -use chrono::NaiveDate; +use chrono::{NaiveDate, Utc}; use clokwerk::{AsyncScheduler, Interval, Job}; use futures::TryStreamExt; use futures_util::TryFutureExt; @@ -119,7 +119,7 @@ impl HotTierManager { if human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( format!( - "Total size required to download the files - {}, size provided for the hot tier - {}, not enough space in hot tier", + "Total size required to download the files: {}. Provided hot tier size: {}. Not enough space in the hot tier. Please increase the hot tier size.", bytes_to_human_size(total_size_to_download), &stream_hot_tier.size ), @@ -259,6 +259,10 @@ impl HotTierManager { object_store: Arc, ) -> Result<(), HotTierError> { let date_str = date.to_string(); + let available_date_list = self.get_hot_tier_date_list(stream).await?; + if available_date_list.contains(&date) && !date.eq(&Utc::now().date_naive()) { + return Ok(()); + } let manifest_files_to_download = s3_file_list .iter() .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) @@ -301,14 +305,11 @@ impl HotTierManager { let parquet_path = self.hot_tier_path.join(parquet_file_path); if !parquet_path.exists() { - fs::create_dir_all(parquet_path.parent().unwrap()).await?; let parquet_file_path = RelativePathBuf::from(parquet_file_path); - if human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() <= parquet_file.file_size { - let date_list = - self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let date_list = self.get_hot_tier_date_list(stream).await?; let date_to_delete = vec![*date_list.first().unwrap()]; self.delete_from_hot_tier(stream_hot_tier, stream, &date_to_delete, false) .await?; @@ -347,6 +348,15 @@ impl HotTierManager { if path.exists() { if !validate { let size = get_dir_size(path.clone())?; + if self + .check_current_date_for_deletion(stream_hot_tier, stream) + .await? + { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( + format!( + "Hot tier capacity for stream {} is exhausted (Total: {}, Available - {})Today's data cannot be deleted. Please increase the hot tier size. Download will resume tomorrow" + , stream, stream_hot_tier.size, stream_hot_tier.available_size.clone().unwrap()) ))); + } stream_hot_tier.used_size = Some(bytes_to_human_size( human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap() - size, @@ -364,6 +374,20 @@ impl HotTierManager { Ok(()) } + async fn check_current_date_for_deletion( + &self, + stream_hot_tier: &StreamHotTier, + stream: &str, + ) -> Result { + let current_date = Utc::now().date_naive(); + let (_, end_date) = + parse_human_date(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let is_end_date_today = end_date == current_date; + let available_date_list = self.get_hot_tier_date_list(stream).await?; + let is_current_date_available = available_date_list.contains(¤t_date); + Ok(available_date_list.len() == 1 && is_current_date_available && is_end_date_today) + } + pub async fn get_hot_tier_date_list( &self, stream: &str, @@ -414,6 +438,7 @@ impl HotTierManager { .all(|file| !file.file_path.eq(&manifest_file.file_path)) }) .collect(); + Ok((hot_tier_files, remaining_files)) } From 691aec18414cb1da272c248cd029a33c0c3f4367 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Thu, 25 Jul 2024 20:19:46 +0530 Subject: [PATCH 05/12] fix: update hot tier in case of no manifest for a particular date in the date range remove date from the range --- server/src/handlers/http/logstream.rs | 5 +- server/src/hottier.rs | 144 ++++++++++++++------- server/src/query/stream_schema_provider.rs | 2 +- 3 files changed, 103 insertions(+), 48 deletions(-) diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 34e8a3e3b..8d99b9b5c 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -964,18 +964,19 @@ pub async fn put_stream_hot_tier( STREAM_INFO.set_hot_tier(&stream_name, true)?; if let Some(hot_tier_manager) = HotTierManager::global() { - let hottier = StreamHotTier { + let mut hottier = StreamHotTier { start_date: hottier.start_date, end_date: hottier.end_date, size: hottier.size.clone(), used_size: Some("0GiB".to_string()), available_size: Some(hottier.size), + updated_date_range: None, }; hot_tier_manager.validate(&stream_name, &hottier).await?; hot_tier_manager - .put_hot_tier(&stream_name, &hottier) + .put_hot_tier(&stream_name, &mut hottier) .await?; let storage = CONFIG.storage().get_object_store(); let mut stream_metadata = storage.get_object_store_format(&stream_name).await?; diff --git a/server/src/hottier.rs b/server/src/hottier.rs index b422a3f68..80e80762e 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -58,6 +58,8 @@ pub struct StreamHotTier { pub start_date: String, #[serde(rename = "end_date")] pub end_date: String, + #[serde(skip_serializing_if = "Option::is_none")] + pub updated_date_range: Option>, } pub struct HotTierManager { @@ -86,8 +88,11 @@ impl HotTierManager { stream: &str, stream_hot_tier: &StreamHotTier, ) -> Result<(), HotTierError> { - let date_list = - self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let date_list = self.get_date_list( + &stream_hot_tier.start_date, + &stream_hot_tier.end_date, + &None, + )?; let object_store = CONFIG.storage().get_object_store(); let s3_file_list = object_store.list_files(stream).await?; let mut manifest_list = Vec::new(); @@ -100,8 +105,6 @@ impl HotTierManager { .collect::>(); for file in manifest_files_to_download { - let path = self.hot_tier_path.join(file); - fs::create_dir_all(path.parent().unwrap()).await?; let manifest_path: RelativePathBuf = RelativePathBuf::from(file); let manifest_file = object_store.get_object(&manifest_path).await?; @@ -119,16 +122,21 @@ impl HotTierManager { if human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( format!( - "Total size required to download the files: {}. Provided hot tier size: {}. Not enough space in the hot tier. Please increase the hot tier size.", + "Total size required to download the files: {}. Provided hot tier size: {}. Not enough space in the hot tier. Please increase the hot tier size and try again.", bytes_to_human_size(total_size_to_download), &stream_hot_tier.size ), ))); } if let Ok(mut existing_hot_tier) = self.get_hot_tier(stream).await { - let available_date_list = self.get_hot_tier_date_list(stream).await?; - self.delete_from_hot_tier(&mut existing_hot_tier, stream, &available_date_list, true) - .await?; + let available_date_list = self.fetch_hot_tier_dates(stream).await?; + self.delete_files_from_hot_tier( + &mut existing_hot_tier, + stream, + &available_date_list, + true, + ) + .await?; } Ok(()) @@ -155,8 +163,21 @@ impl HotTierManager { pub async fn put_hot_tier( &self, stream: &str, - hot_tier: &StreamHotTier, + hot_tier: &mut StreamHotTier, ) -> Result<(), HotTierError> { + let date_list = if let Some(updated_date_range) = &hot_tier.updated_date_range { + updated_date_range + .iter() + .map(|date| NaiveDate::parse_from_str(date, "%Y-%m-%d").unwrap()) + .collect() + } else { + self.get_date_list(&hot_tier.start_date, &hot_tier.end_date, &None)? + }; + hot_tier.updated_date_range = date_list + .iter() + .map(|date| date.to_string()) + .collect::>() + .into(); let path = hot_tier_file_path(&self.hot_tier_path, stream)?; let bytes = serde_json::to_vec(hot_tier)?.into(); self.filesystem.put(&path, bytes).await?; @@ -200,19 +221,16 @@ impl HotTierManager { let mut stream_hot_tier = self.get_hot_tier(&stream).await?; let mut parquet_file_size = human_size_to_bytes(stream_hot_tier.used_size.as_ref().unwrap()).unwrap(); - let date_list = - self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; - let available_date_list = self.get_hot_tier_date_list(&stream).await?; + let date_list = self.get_hot_tier_time_range(&stream_hot_tier).await?; + let available_date_list = self.fetch_hot_tier_dates(&stream).await?; let dates_to_delete: Vec = available_date_list .into_iter() .filter(|available_date| !date_list.contains(available_date)) .collect(); - if !dates_to_delete.is_empty() { - self.delete_from_hot_tier(&mut stream_hot_tier, &stream, &dates_to_delete, false) + self.delete_files_from_hot_tier(&mut stream_hot_tier, &stream, &dates_to_delete, false) .await?; } - let object_store = CONFIG.storage().get_object_store(); let s3_file_list = object_store.list_files(&stream).await?; @@ -226,7 +244,7 @@ impl HotTierManager { object_store.clone(), ) .await?; - self.put_hot_tier(&stream, &stream_hot_tier).await?; + self.put_hot_tier(&stream, &mut stream_hot_tier).await?; } Ok(()) @@ -236,15 +254,21 @@ impl HotTierManager { &self, start_date: &str, end_date: &str, + updated_date_range: &Option>, ) -> Result, HotTierError> { - let (start_date, end_date) = parse_human_date(start_date, end_date)?; - let mut date_list = Vec::new(); - let mut current_date = start_date; - - while current_date <= end_date { - date_list.push(current_date); - current_date += chrono::Duration::days(1); - } + let (dt_start_date, dt_end_date) = if let Some(updated_date_range) = updated_date_range { + parse_human_date( + updated_date_range.first().unwrap(), + updated_date_range.last().unwrap(), + )? + } else { + parse_human_date(start_date, end_date)? + }; + + let date_list: Vec = (0..) + .map(|i| dt_start_date + chrono::Duration::days(i)) + .take_while(|&date| date <= dt_end_date) + .collect(); Ok(date_list) } @@ -259,7 +283,7 @@ impl HotTierManager { object_store: Arc, ) -> Result<(), HotTierError> { let date_str = date.to_string(); - let available_date_list = self.get_hot_tier_date_list(stream).await?; + let available_date_list = self.fetch_hot_tier_dates(stream).await?; if available_date_list.contains(&date) && !date.eq(&Utc::now().date_naive()) { return Ok(()); } @@ -267,6 +291,10 @@ impl HotTierManager { .iter() .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) .collect::>(); + if manifest_files_to_download.is_empty() { + self.update_hot_tier_time_range(stream, stream_hot_tier, &date_str) + .await?; + } let mut manifest_list = Vec::new(); for file in manifest_files_to_download { let path = self.hot_tier_path.join(file); @@ -309,11 +337,10 @@ impl HotTierManager { if human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() <= parquet_file.file_size { - let date_list = self.get_hot_tier_date_list(stream).await?; + let date_list = self.fetch_hot_tier_dates(stream).await?; let date_to_delete = vec![*date_list.first().unwrap()]; - self.delete_from_hot_tier(stream_hot_tier, stream, &date_to_delete, false) + self.delete_files_from_hot_tier(stream_hot_tier, stream, &date_to_delete, false) .await?; - self.update_hot_tier(stream_hot_tier).await?; *parquet_file_size = human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap(); } @@ -336,7 +363,7 @@ impl HotTierManager { Ok(()) } - pub async fn delete_from_hot_tier( + pub async fn delete_files_from_hot_tier( &self, stream_hot_tier: &mut StreamHotTier, stream: &str, @@ -369,6 +396,8 @@ impl HotTierManager { } fs::remove_dir_all(path.clone()).await?; } + self.update_hot_tier_time_range(stream, stream_hot_tier, &date.to_string()) + .await?; } Ok(()) @@ -383,15 +412,12 @@ impl HotTierManager { let (_, end_date) = parse_human_date(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; let is_end_date_today = end_date == current_date; - let available_date_list = self.get_hot_tier_date_list(stream).await?; + let available_date_list = self.fetch_hot_tier_dates(stream).await?; let is_current_date_available = available_date_list.contains(¤t_date); Ok(available_date_list.len() == 1 && is_current_date_available && is_end_date_today) } - pub async fn get_hot_tier_date_list( - &self, - stream: &str, - ) -> Result, HotTierError> { + pub async fn fetch_hot_tier_dates(&self, stream: &str) -> Result, HotTierError> { let mut date_list = Vec::new(); let path = self.hot_tier_path.join(stream); if path.exists() { @@ -408,23 +434,53 @@ impl HotTierManager { ); } } + date_list.sort(); Ok(date_list) } - pub async fn update_hot_tier( + pub async fn update_hot_tier_time_range( &self, + stream: &str, stream_hot_tier: &mut StreamHotTier, + date: &str, ) -> Result<(), HotTierError> { - let start_date = &stream_hot_tier.start_date; - let end_date = &stream_hot_tier.end_date; - let mut date_list = self.get_date_list(start_date, end_date)?; - - date_list.retain(|date: &NaiveDate| *date.to_string() != *start_date); - stream_hot_tier.start_date = date_list.first().unwrap().to_string(); + let mut existing_date_range = stream_hot_tier.updated_date_range.as_ref().unwrap().clone(); + existing_date_range.retain(|d| d != date); + stream_hot_tier.updated_date_range = Some(existing_date_range); + self.put_hot_tier(stream, stream_hot_tier).await?; Ok(()) } - pub async fn get_hot_tier_manifests( + pub async fn get_hot_tier_time_range( + &self, + stream_hot_tier: &StreamHotTier, + ) -> Result, HotTierError> { + let (start_date, end_date) = + parse_human_date(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let date_list: Vec = (0..) + .map(|i| start_date + chrono::Duration::days(i)) + .take_while(|&date| date <= end_date) + .collect(); + let mut existing_date_range: Vec = stream_hot_tier + .updated_date_range + .as_ref() + .unwrap() + .iter() + .map(|date| NaiveDate::parse_from_str(date, "%Y-%m-%d").unwrap()) + .collect(); + existing_date_range.sort(); + let mut updated_date_range = vec![*date_list.last().unwrap()]; + updated_date_range.extend( + date_list + .into_iter() + .filter(|date| existing_date_range.contains(date)), + ); + updated_date_range.sort(); + updated_date_range.dedup(); + Ok(updated_date_range) + } + + pub async fn get_hot_tier_manifest_files( &self, stream: &str, manifest_files: Vec, @@ -447,9 +503,7 @@ impl HotTierManager { stream: &str, ) -> Result, HotTierError> { let mut hot_tier_parquet_files: Vec = Vec::new(); - let stream_hot_tier = self.get_hot_tier(stream).await?; - let date_list = - self.get_date_list(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; + let date_list = self.fetch_hot_tier_dates(stream).await?; for date in date_list { let date_str = date.to_string(); let path = &self diff --git a/server/src/query/stream_schema_provider.rs b/server/src/query/stream_schema_provider.rs index d9e94ece7..98d097c10 100644 --- a/server/src/query/stream_schema_provider.rs +++ b/server/src/query/stream_schema_provider.rs @@ -422,7 +422,7 @@ impl TableProvider for StandardTableProvider { if let Some(hot_tier_manager) = HotTierManager::global() { if hot_tier_manager.check_stream_hot_tier_exists(&self.stream) { let (hot_tier_files, remainder) = hot_tier_manager - .get_hot_tier_manifests(&self.stream, manifest_files) + .get_hot_tier_manifest_files(&self.stream, manifest_files) .await .map_err(|err| DataFusionError::External(Box::new(err)))?; // Assign remaining entries back to manifest list From e285c338c3af55d5a53e3e9ba5f09d593d3fc345 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Mon, 29 Jul 2024 12:36:16 +0530 Subject: [PATCH 06/12] enhancement of hot tier implementation - check for disk availability, - delete older files from hot tier if disk usage exceeds threshold - download the latest entries from S3 to hot tier --- server/src/cli.rs | 38 ++- server/src/handlers/http/logstream.rs | 14 +- server/src/hottier.rs | 440 ++++++++++++++++++++------ server/src/option.rs | 21 +- server/src/utils.rs | 20 +- server/src/validator.rs | 18 +- 6 files changed, 444 insertions(+), 107 deletions(-) diff --git a/server/src/cli.rs b/server/src/cli.rs index 3787fb721..e451c1ab1 100644 --- a/server/src/cli.rs +++ b/server/src/cli.rs @@ -104,6 +104,12 @@ pub struct Cli { /// The local hot_tier path is used for optimising the query performance in the distributed systems pub hot_tier_storage_path: Option, + + /// Size for local cache + pub min_hot_tier_size: u64, + + ///maximum disk usage for hot tier + pub max_disk_usage: f64, } impl Cli { @@ -138,6 +144,8 @@ impl Cli { pub const FLIGHT_PORT: &'static str = "flight-port"; pub const CORS: &'static str = "cors"; pub const HOT_TIER_PATH: &'static str = "hot-tier-path"; + pub const MIN_HOT_TIER_SIZE: &'static str = "hot-tier-size"; + pub const MAX_DISK_USAGE: &'static str = "max-disk-usage"; pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) @@ -408,7 +416,27 @@ impl Cli { .value_parser(validation::canonicalize_path) .help("Local path on this device to be used for hot tier data") .next_line_help(true), - ).group( + ) + .arg( + Arg::new(Self::MIN_HOT_TIER_SIZE) + .long(Self::MIN_HOT_TIER_SIZE) + .env("P_MIN_HOT_TIER_SIZE") + .value_name("size") + .default_value("100GiB") + .value_parser(validation::hot_tier_size) + .help("Minimum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") + .next_line_help(true), + ) + .arg( + Arg::new(Self::MAX_DISK_USAGE) + .long(Self::MAX_DISK_USAGE) + .env("P_MAX_DISK_USAGE") + .value_name("size") + .default_value("80.0") + .value_parser(validation::disk_usage) + .help("Maximum disk usage for hot tier"), + ) + .group( ArgGroup::new("oidc") .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) .requires_all([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) @@ -546,7 +574,15 @@ impl FromArgMatches for Cli { }; self.hot_tier_storage_path = m.get_one::(Self::HOT_TIER_PATH).cloned(); + self.min_hot_tier_size = m + .get_one::(Self::MIN_HOT_TIER_SIZE) + .cloned() + .expect("default value for cache size"); + self.max_disk_usage = m + .get_one::(Self::MAX_DISK_USAGE) + .cloned() + .expect("default value for max disk usage"); Ok(()) } } diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 8d99b9b5c..0d7b12fec 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -951,7 +951,7 @@ pub async fn put_stream_hot_tier( } let body = body.into_inner(); - let hottier: StreamHotTier = match serde_json::from_value(body) { + let mut hottier: StreamHotTier = match serde_json::from_value(body) { Ok(hottier) => hottier, Err(err) => return Err(StreamError::InvalidHotTierConfig(err)), }; @@ -964,7 +964,7 @@ pub async fn put_stream_hot_tier( STREAM_INFO.set_hot_tier(&stream_name, true)?; if let Some(hot_tier_manager) = HotTierManager::global() { - let mut hottier = StreamHotTier { + hottier = StreamHotTier { start_date: hottier.start_date, end_date: hottier.end_date, size: hottier.size.clone(), @@ -973,8 +973,9 @@ pub async fn put_stream_hot_tier( updated_date_range: None, }; - hot_tier_manager.validate(&stream_name, &hottier).await?; - + hot_tier_manager + .validate(&stream_name, &mut hottier) + .await?; hot_tier_manager .put_hot_tier(&stream_name, &mut hottier) .await?; @@ -986,10 +987,7 @@ pub async fn put_stream_hot_tier( .await?; } - Ok(( - format!("hot tier set for log stream {stream_name}"), - StatusCode::OK, - )) + Ok((web::Json(hottier), StatusCode::OK)) } pub async fn get_stream_hot_tier(req: HttpRequest) -> Result { diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 80e80762e..2bddd6011 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -16,7 +16,11 @@ * */ -use std::{io, path::PathBuf, sync::Arc}; +use std::{ + io, + path::{Path, PathBuf}, + sync::Arc, +}; use crate::{ catalog::manifest::{File, Manifest}, @@ -26,7 +30,7 @@ use crate::{ CONFIG, }, storage::{ObjectStorage, ObjectStorageError}, - utils::get_dir_size, + utils::{extract_datetime, get_dir_size}, validator::{error::HotTierValidationError, parse_human_date}, }; use chrono::{NaiveDate, Utc}; @@ -38,6 +42,7 @@ use once_cell::sync::OnceCell; use parquet::errors::ParquetError; use relative_path::RelativePathBuf; use std::time::Duration; +use sysinfo::{Disks, System}; use tokio::fs::{self, DirEntry}; use tokio::io::AsyncWriteExt; use tokio_stream::wrappers::ReadDirStream; @@ -83,65 +88,86 @@ impl HotTierManager { })) } + ///validate the hot tier configuration, check the disk availability + /// update the hot tier range if the disk is not available to download the entire set + /// delete the existing hot tier files once validation is successful pub async fn validate( &self, stream: &str, - stream_hot_tier: &StreamHotTier, + stream_hot_tier: &mut StreamHotTier, ) -> Result<(), HotTierError> { - let date_list = self.get_date_list( - &stream_hot_tier.start_date, - &stream_hot_tier.end_date, - &None, - )?; - let object_store = CONFIG.storage().get_object_store(); - let s3_file_list = object_store.list_files(stream).await?; - let mut manifest_list = Vec::new(); - let mut total_size_to_download = 0; - for date in date_list { - let date_str = date.to_string(); - let manifest_files_to_download = s3_file_list - .iter() - .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) - .collect::>(); + async fn validate_helper( + this: &HotTierManager, + stream: &str, + stream_hot_tier: &mut StreamHotTier, + ) -> Result<(), HotTierError> { + let date_list = this.get_date_list( + &stream_hot_tier.start_date, + &stream_hot_tier.end_date, + &stream_hot_tier.updated_date_range, + )?; + let object_store = CONFIG.storage().get_object_store(); + let s3_file_list = object_store.list_files(stream).await?; + let mut manifest_list = Vec::new(); + let mut total_size_to_download = 0; + + for date in &date_list { + let date_str = date.to_string(); + let manifest_files_to_download = s3_file_list + .iter() + .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) + .collect::>(); + + for file in manifest_files_to_download { + let manifest_path: RelativePathBuf = RelativePathBuf::from(file); + let manifest_file = object_store.get_object(&manifest_path).await?; + let manifest: Manifest = serde_json::from_slice(&manifest_file)?; + manifest_list.push(manifest.clone()); + } + } - for file in manifest_files_to_download { - let manifest_path: RelativePathBuf = RelativePathBuf::from(file); - let manifest_file = object_store.get_object(&manifest_path).await?; + for manifest in &manifest_list { + total_size_to_download += manifest + .files + .iter() + .map(|file| file.file_size) + .sum::(); + } - let manifest: Manifest = serde_json::from_slice(&manifest_file)?; - manifest_list.push(manifest.clone()); + if (!this + .is_disk_available(total_size_to_download, &stream_hot_tier.size) + .await? + || human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download) + && date_list.len() > 1 + { + stream_hot_tier.updated_date_range = Some( + date_list + .iter() + .skip(1) + .map(|date| date.to_string()) + .collect(), + ); + return Box::pin(validate_helper(this, stream, stream_hot_tier)).await; } - } - for manifest in &manifest_list { - total_size_to_download += manifest - .files - .iter() - .map(|file| file.file_size) - .sum::(); - } - if human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download { - return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( - format!( - "Total size required to download the files: {}. Provided hot tier size: {}. Not enough space in the hot tier. Please increase the hot tier size and try again.", - bytes_to_human_size(total_size_to_download), - &stream_hot_tier.size - ), - ))); - } - if let Ok(mut existing_hot_tier) = self.get_hot_tier(stream).await { - let available_date_list = self.fetch_hot_tier_dates(stream).await?; - self.delete_files_from_hot_tier( - &mut existing_hot_tier, - stream, - &available_date_list, - true, - ) - .await?; + + if let Ok(mut existing_hot_tier) = this.get_hot_tier(stream).await { + let available_date_list = this.fetch_hot_tier_dates(stream).await?; + this.delete_files_from_hot_tier( + &mut existing_hot_tier, + stream, + &available_date_list, + true, + ) + .await?; + } + + Ok(()) } - Ok(()) + validate_helper(self, stream, stream_hot_tier).await } + ///get the hot tier metadata file for the stream pub async fn get_hot_tier(&self, stream: &str) -> Result { if !self.check_stream_hot_tier_exists(stream) { return Err(HotTierError::HotTierValidationError( @@ -160,6 +186,8 @@ impl HotTierManager { } } + ///put the hot tier metadata file for the stream + /// set the updated_date_range in the hot tier metadata file pub async fn put_hot_tier( &self, stream: &str, @@ -184,6 +212,7 @@ impl HotTierManager { Ok(()) } + ///schedule the download of the hot tier files from S3 every minute pub fn download_from_s3<'a>(&'a self) -> Result<(), HotTierError> where 'a: 'static, @@ -207,6 +236,7 @@ impl HotTierManager { Ok(()) } + ///sync the hot tier files from S3 to the hot tier directory for all streams async fn sync_hot_tier(&self) -> Result<(), HotTierError> { let streams = STREAM_INFO.list_streams(); for stream in streams { @@ -217,6 +247,8 @@ impl HotTierManager { Ok(()) } + ///process the hot tier files for the stream + /// delete the files from the hot tier directory if the available date range is outside the hot tier range async fn process_stream(&self, stream: String) -> Result<(), HotTierError> { let mut stream_hot_tier = self.get_hot_tier(&stream).await?; let mut parquet_file_size = @@ -250,6 +282,9 @@ impl HotTierManager { Ok(()) } + ///get the list of date range from hot tier metadata + /// if updated_date_range is available, get the list of dates from the updated_date_range + /// else get the list of dates from the start and end date fn get_date_list( &self, start_date: &str, @@ -273,6 +308,10 @@ impl HotTierManager { Ok(date_list) } + ///process the hot tier files for the date for the stream + /// collect all manifests from S3 for the date, sort the parquet file list + /// in order to download the latest files first + /// download the parquet files if not present in hot tier directory async fn process_date( &self, stream: &str, @@ -307,62 +346,93 @@ impl HotTierManager { let manifest: Manifest = serde_json::from_slice(&manifest_file)?; manifest_list.push(manifest.clone()); } - + let mut parquet_file_list = Vec::new(); for manifest in manifest_list { - for parquet_file in manifest.files { - self.process_parquet_file( - stream, - stream_hot_tier, - &parquet_file, - parquet_file_size, - ) - .await?; + parquet_file_list.extend(manifest.files); + } + parquet_file_list.sort_by_key(|file| file.file_path.clone()); + parquet_file_list.reverse(); + for parquet_file in parquet_file_list { + let parquet_file_path = &parquet_file.file_path; + let parquet_path = self.hot_tier_path.join(parquet_file_path); + if !parquet_path.exists() { + if !self + .process_parquet_file( + stream, + stream_hot_tier, + &parquet_file, + parquet_file_size, + parquet_path, + ) + .await? + { + break; + } + } else { + continue; } } + Ok(()) } + ///process the parquet file for the stream + /// check if the disk is available to download the parquet file + /// if not available, delete the oldest entry from the hot tier directory + /// download the parquet file from S3 to the hot tier directory + /// update the used and available size in the hot tier metadata + /// return true if the parquet file is processed successfully async fn process_parquet_file( &self, stream: &str, stream_hot_tier: &mut StreamHotTier, parquet_file: &File, parquet_file_size: &mut u64, - ) -> Result<(), HotTierError> { - let parquet_file_path = &parquet_file.file_path; - let parquet_path = self.hot_tier_path.join(parquet_file_path); - - if !parquet_path.exists() { - let parquet_file_path = RelativePathBuf::from(parquet_file_path); - if human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() + parquet_path: PathBuf, + ) -> Result { + let mut file_processed = false; + if !self + .is_disk_available(parquet_file.file_size, &stream_hot_tier.size) + .await? + || human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() <= parquet_file.file_size + { + if !self + .delete_1minute_data( + stream, + stream_hot_tier, + &parquet_path, + parquet_file.file_size, + ) + .await? { - let date_list = self.fetch_hot_tier_dates(stream).await?; - let date_to_delete = vec![*date_list.first().unwrap()]; - self.delete_files_from_hot_tier(stream_hot_tier, stream, &date_to_delete, false) - .await?; - *parquet_file_size = - human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap(); + return Ok(file_processed); } - - fs::create_dir_all(parquet_path.parent().unwrap()).await?; - let mut file = fs::File::create(parquet_path.clone()).await?; - let parquet_data = CONFIG - .storage() - .get_object_store() - .get_object(&parquet_file_path) - .await?; - file.write_all(&parquet_data).await?; - *parquet_file_size += parquet_file.file_size; - stream_hot_tier.used_size = Some(bytes_to_human_size(*parquet_file_size)); - stream_hot_tier.available_size = Some(bytes_to_human_size( - human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() - - parquet_file.file_size, - )); + *parquet_file_size = + human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap(); } - Ok(()) + let parquet_file_path = RelativePathBuf::from(parquet_file.file_path.clone()); + fs::create_dir_all(parquet_path.parent().unwrap()).await?; + let mut file = fs::File::create(parquet_path.clone()).await?; + let parquet_data = CONFIG + .storage() + .get_object_store() + .get_object(&parquet_file_path) + .await?; + file.write_all(&parquet_data).await?; + *parquet_file_size += parquet_file.file_size; + stream_hot_tier.used_size = Some(bytes_to_human_size(*parquet_file_size)); + + stream_hot_tier.available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() + - parquet_file.file_size, + )); + file_processed = true; + Ok(file_processed) } + ///delete the files for the date range given from the hot tier directory for the stream + /// update the used and available size in the hot tier metadata pub async fn delete_files_from_hot_tier( &self, stream_hot_tier: &mut StreamHotTier, @@ -381,7 +451,7 @@ impl HotTierManager { { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( format!( - "Hot tier capacity for stream {} is exhausted (Total: {}, Available - {})Today's data cannot be deleted. Please increase the hot tier size. Download will resume tomorrow" + "Hot tier capacity for stream {} is exhausted (Total: {}, Available - {}). Today's data cannot be deleted. Please increase the hot tier size. Download will resume tomorrow" , stream, stream_hot_tier.size, stream_hot_tier.available_size.clone().unwrap()) ))); } stream_hot_tier.used_size = Some(bytes_to_human_size( @@ -393,6 +463,7 @@ impl HotTierManager { .unwrap() + size, )); + self.put_hot_tier(stream, stream_hot_tier).await?; } fs::remove_dir_all(path.clone()).await?; } @@ -403,6 +474,7 @@ impl HotTierManager { Ok(()) } + ///check if the current date is the last date available in the hot tier directory async fn check_current_date_for_deletion( &self, stream_hot_tier: &StreamHotTier, @@ -417,6 +489,7 @@ impl HotTierManager { Ok(available_date_list.len() == 1 && is_current_date_available && is_end_date_today) } + ///fetch the list of dates available in the hot tier directory for the stream and sort them pub async fn fetch_hot_tier_dates(&self, stream: &str) -> Result, HotTierError> { let mut date_list = Vec::new(); let path = self.hot_tier_path.join(stream); @@ -438,19 +511,35 @@ impl HotTierManager { Ok(date_list) } + ///update the hot tier time range for the stream + /// remove the date given from the updated_date_range + /// put the hot tier metadata file for the stream pub async fn update_hot_tier_time_range( &self, stream: &str, stream_hot_tier: &mut StreamHotTier, date: &str, ) -> Result<(), HotTierError> { + if stream_hot_tier.updated_date_range.as_ref().unwrap().len() == 1 + && stream_hot_tier + .updated_date_range + .as_ref() + .unwrap() + .contains(&Utc::now().date_naive().to_string()) + { + return Ok(()); + } let mut existing_date_range = stream_hot_tier.updated_date_range.as_ref().unwrap().clone(); existing_date_range.retain(|d| d != date); stream_hot_tier.updated_date_range = Some(existing_date_range); + self.put_hot_tier(stream, stream_hot_tier).await?; Ok(()) } + ///get the updated hot tier time range for the stream + /// get the existing dates from the updated_date_range + /// update the updated_date_range with the existing dates and the new dates pub async fn get_hot_tier_time_range( &self, stream_hot_tier: &StreamHotTier, @@ -480,6 +569,7 @@ impl HotTierManager { Ok(updated_date_range) } + ///get the list of files from all the manifests present in hot tier directory for the stream pub async fn get_hot_tier_manifest_files( &self, stream: &str, @@ -498,6 +588,7 @@ impl HotTierManager { Ok((hot_tier_files, remaining_files)) } + ///get the list of parquet files from the hot tier directory for the stream pub async fn get_hot_tier_parquet_files( &self, stream: &str, @@ -534,6 +625,8 @@ impl HotTierManager { } Ok(hot_tier_parquet_files) } + + ///check if the hot tier metadata file exists for the stream pub fn check_stream_hot_tier_exists(&self, stream: &str) -> bool { let path = self .hot_tier_path @@ -541,8 +634,151 @@ impl HotTierManager { .join(STREAM_HOT_TIER_FILENAME); path.exists() } + + ///delete the parquet file from the hot tier directory for the stream + /// loop through all manifests in the hot tier directory for the stream + /// loop through all parquet files in the manifest + /// check for the oldest entry to delete if the path exists in hot tier + /// update the used and available size in the hot tier metadata + /// loop if available size is still less than the parquet file size + pub async fn delete_1minute_data( + &self, + stream: &str, + stream_hot_tier: &mut StreamHotTier, + download_file_path: &Path, + parquet_file_size: u64, + ) -> Result { + let mut delete_successful = false; + let date_list = self.get_date_list( + &stream_hot_tier.start_date, + &stream_hot_tier.end_date, + &stream_hot_tier.updated_date_range, + )?; + + if let Some(date_to_delete) = date_list.first() { + let date_str = date_to_delete.to_string(); + let path = self + .hot_tier_path + .join(stream) + .join(format!("date={}", date_str)); + + if !path.exists() { + return Ok(delete_successful); + } + + let date_dirs = ReadDirStream::new(fs::read_dir(&path).await?); + let mut manifest_files: Vec = date_dirs.try_collect().await?; + manifest_files.retain(|file| { + file.file_name() + .to_string_lossy() + .ends_with("manifest.json") + }); + 'loop_manifests: for manifest_file in manifest_files { + let file = fs::read(manifest_file.path()).await?; + let mut manifest: Manifest = serde_json::from_slice(&file)?; + + manifest.files.sort_by_key(|file| file.file_path.clone()); + manifest.files.reverse(); + + 'loop_files: while let Some(file_to_delete) = manifest.files.pop() { + let file_size = file_to_delete.file_size; + let path_to_delete = CONFIG + .parseable + .hot_tier_storage_path + .as_ref() + .unwrap() + .join(&file_to_delete.file_path); + + if path_to_delete.exists() { + if let (Some(download_date_time), Some(delete_date_time)) = ( + extract_datetime(download_file_path.to_str().unwrap()), + extract_datetime(path_to_delete.to_str().unwrap()), + ) { + if download_date_time <= delete_date_time { + delete_successful = false; + break 'loop_files; + } + } + + fs::write(manifest_file.path(), serde_json::to_vec(&manifest)?).await?; + fs::remove_dir_all(&path_to_delete.parent().unwrap()).await?; + + stream_hot_tier.used_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()) + .unwrap() + - file_size, + )); + stream_hot_tier.available_size = Some(bytes_to_human_size( + human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()) + .unwrap() + + file_size, + )); + self.put_hot_tier(stream, stream_hot_tier).await?; + delete_successful = true; + + if human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()) + .unwrap() + <= parquet_file_size + { + continue 'loop_files; + } else { + break 'loop_manifests; + } + } else { + fs::write(manifest_file.path(), serde_json::to_vec(&manifest)?).await?; + } + } + } + } + Ok(delete_successful) + } + + ///check if the disk is available to download the parquet file + /// check if the disk usage is above the threshold + pub async fn is_disk_available( + &self, + size_to_download: u64, + hot_tier_size: &str, + ) -> Result { + let (total_disk_space, available_disk_space, used_disk_space) = get_disk_usage(); + + if let Some(available_space) = available_disk_space { + let hot_tier_size = human_size_to_bytes(hot_tier_size).unwrap(); + if available_space < hot_tier_size { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( + "Not enough space left in the disk for hot tier. Disk available Size: {}, Hot Tier Size: {}", + bytes_to_human_size(available_space), + bytes_to_human_size(hot_tier_size) + )))); + } + + if used_disk_space.unwrap() as f64 * 100.0 / total_disk_space.unwrap() as f64 + > CONFIG.parseable.max_disk_usage + { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( + "Disk usage is above the threshold. Disk Used Size: {}, Disk Total Size: {}, Disk Usage {}%", + bytes_to_human_size(used_disk_space.unwrap()), + bytes_to_human_size(total_disk_space.unwrap()), + used_disk_space.unwrap() as f64 * 100.0 / total_disk_space.unwrap() as f64 + )))); + } + if available_space < size_to_download { + return Ok(false); + } + + if ((used_disk_space.unwrap() + size_to_download) as f64 * 100.0 + / total_disk_space.unwrap() as f64) + > CONFIG.parseable.max_disk_usage + { + return Ok(false); + } + } + + Ok(true) + } } +/// get the hot tier file path for the stream pub fn hot_tier_file_path( root: impl AsRef, stream: &str, @@ -551,6 +787,32 @@ pub fn hot_tier_file_path( object_store::path::Path::from_absolute_path(path) } +///get the disk usage for the hot tier storage path +pub fn get_disk_usage() -> (Option, Option, Option) { + let mut sys = System::new_all(); + sys.refresh_all(); + let path = CONFIG.parseable.hot_tier_storage_path.as_ref().unwrap(); + + let mut disks = Disks::new_with_refreshed_list(); + disks.sort_by_key(|disk| disk.mount_point().to_str().unwrap().len()); + disks.reverse(); + + for disk in disks.iter() { + if path.starts_with(disk.mount_point().to_str().unwrap()) { + let total_disk_space = disk.total_space(); + let available_disk_space = disk.available_space(); + let used_disk_space = total_disk_space - available_disk_space; + return ( + Some(total_disk_space), + Some(available_disk_space), + Some(used_disk_space), + ); + } + } + + (None, None, None) +} + #[derive(Debug, thiserror::Error)] pub enum HotTierError { #[error("{0}")] diff --git a/server/src/option.rs b/server/src/option.rs index 01b75a567..07be94f90 100644 --- a/server/src/option.rs +++ b/server/src/option.rs @@ -28,7 +28,7 @@ use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; use std::env; use std::path::PathBuf; use std::sync::Arc; -pub const MIN_CACHE_SIZE_BYTES: u64 = 1000u64.pow(3); // 1 GiB +pub const MIN_CACHE_SIZE_BYTES: u64 = 1073741824; // 1 GiB pub const JOIN_COMMUNITY: &str = "Join us on Parseable Slack community for questions : https://logg.ing/community"; pub static CONFIG: Lazy> = Lazy::new(|| Arc::new(Config::new())); @@ -342,7 +342,6 @@ pub mod validation { .or(parse_and_map::(s)) .or(parse_and_map::(s)) .map_err(|_| "Could not parse given size".to_string())?; - Ok(size) } @@ -369,6 +368,17 @@ pub mod validation { } pub fn cache_size(s: &str) -> Result { + let size = human_size_to_bytes(s)?; + if size < MIN_CACHE_SIZE_BYTES { + return Err(format!( + "Specified value of cache size is smaller than current minimum of {}", + human_size_to_bytes(&MIN_CACHE_SIZE_BYTES.to_string()).unwrap() + )); + } + Ok(size) + } + + pub fn hot_tier_size(s: &str) -> Result { let size = human_size_to_bytes(s)?; if size < MIN_CACHE_SIZE_BYTES { return Err( @@ -377,4 +387,11 @@ pub mod validation { } Ok(size) } + + pub fn disk_usage(s: &str) -> Result { + let disk_usage = s + .parse::() + .map_err(|_| "Invalid disk usage value".to_string())?; + Ok(disk_usage) + } } diff --git a/server/src/utils.rs b/server/src/utils.rs index 8565ada04..b574bbfa8 100644 --- a/server/src/utils.rs +++ b/server/src/utils.rs @@ -23,14 +23,14 @@ pub mod json; pub mod uid; pub mod update; use crate::option::CONFIG; -use chrono::{DateTime, NaiveDate, Timelike, Utc}; +use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Utc}; use itertools::Itertools; +use regex::Regex; use sha2::{Digest, Sha256}; use std::collections::HashMap; use std::path::PathBuf; use std::{env, fs}; use url::Url; - #[allow(dead_code)] pub fn hostname() -> Option { hostname::get() @@ -327,6 +327,22 @@ pub fn get_dir_size(path: PathBuf) -> Result { dir_size_recursive(&path) } +pub fn extract_datetime(path: &str) -> Option { + let re = Regex::new(r"date=(\d{4}-\d{2}-\d{2})/hour=(\d{2})/minute=(\d{2})").unwrap(); + if let Some(caps) = re.captures(path) { + let date_str = caps.get(1)?.as_str(); + let hour_str = caps.get(2)?.as_str(); + let minute_str = caps.get(3)?.as_str(); + + let date = NaiveDate::parse_from_str(date_str, "%Y-%m-%d").ok()?; + let time = + NaiveTime::parse_from_str(&format!("{}:{}", hour_str, minute_str), "%H:%M").ok()?; + Some(NaiveDateTime::new(date, time)) + } else { + None + } +} + #[cfg(test)] mod tests { use chrono::DateTime; diff --git a/server/src/validator.rs b/server/src/validator.rs index f163cb1e1..471a1cd6c 100644 --- a/server/src/validator.rs +++ b/server/src/validator.rs @@ -24,7 +24,8 @@ use crate::alerts::rule::base::{NumericRule, StringRule}; use crate::alerts::rule::{ColumnRule, ConsecutiveNumericRule, ConsecutiveStringRule}; use crate::alerts::{Alerts, Rule}; use crate::handlers::http::cluster::INTERNAL_STREAM_NAME; -use crate::option::validation::{cache_size, human_size_to_bytes}; +use crate::option::validation::{bytes_to_human_size, human_size_to_bytes}; +use crate::option::CONFIG; // Add more sql keywords here in lower case const DENIED_NAMES: &[&str] = &[ @@ -154,9 +155,16 @@ pub fn hot_tier( size: &str, ) -> Result<(), HotTierValidationError> { if human_size_to_bytes(size).is_err() { - return Err(HotTierValidationError::Size); + return Err(HotTierValidationError::Size(bytes_to_human_size( + CONFIG.parseable.min_hot_tier_size, + ))); + } + + if human_size_to_bytes(size).unwrap() < CONFIG.parseable.min_hot_tier_size { + return Err(HotTierValidationError::Size(bytes_to_human_size( + CONFIG.parseable.min_hot_tier_size, + ))); } - cache_size(size).map_err(|_| HotTierValidationError::Size)?; let (start_date, end_date) = parse_human_date(start_date, end_date)?; @@ -241,8 +249,8 @@ pub mod error { #[derive(Debug, thiserror::Error)] pub enum HotTierValidationError { - #[error("Invalid size given for hot tier, please provide size in human readable format, e.g 1GiB, 2GiB")] - Size, + #[error("Invalid size given for hot tier, please provide size in human readable format, e.g 1GiB, 2GiB, minimum {0}")] + Size(String), #[error( "Invalid start date given for hot tier, please provide the date in yyyy-mm-dd format" )] From c7da14956bcd01ddd5c8db901374a9333775f75d Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Mon, 29 Jul 2024 15:30:59 +0530 Subject: [PATCH 07/12] fix for query with hot tier files create list of hot tier files where path matches manifest files remaining hot tier files will be set as remainder to be served from S3 --- server/src/hottier.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 2bddd6011..35bedb8de 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -575,7 +575,12 @@ impl HotTierManager { stream: &str, manifest_files: Vec, ) -> Result<(Vec, Vec), HotTierError> { - let hot_tier_files = self.get_hot_tier_parquet_files(stream).await?; + let mut hot_tier_files = self.get_hot_tier_parquet_files(stream).await?; + hot_tier_files.retain(|file| { + manifest_files + .iter() + .any(|manifest_file| manifest_file.file_path.eq(&file.file_path)) + }); let remaining_files: Vec = manifest_files .into_iter() .filter(|manifest_file| { @@ -584,7 +589,6 @@ impl HotTierManager { .all(|file| !file.file_path.eq(&manifest_file.file_path)) }) .collect(); - Ok((hot_tier_files, remaining_files)) } From 8e3aca8a3fe1172107e1c975b1f5ab96c38946fc Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Wed, 31 Jul 2024 12:53:47 +0530 Subject: [PATCH 08/12] enhancement in hot tier implementation - hot tier size to be set for each stream in the API call PUT /logstream/{logstream}/hottier with the body ` { "size": "20GiB" } ` - validations added 1. max disk usage (current used disk space + hot tier size of all the streams) should not exceed 80% 2. minimum hottier size for each stream is set to 10GiB - download to start from the latest date present in S3 till the hottier size is exhausted - to keep the latest data, delete oldest entry in hottier - maintain one manifest file for hottier for each date - current used size, available size and oldest datetime entry in the hottier can be get from the API call GET /logstream/{logstream}/hottier response body - ` { "size": "5GiB", "used_size": "4.24 GiB", "available_size": "775 MiB", "oldest_date_time_entry": "2024-07-28 21:32:00" } ` --- server/src/cli.rs | 35 -- server/src/handlers/http/logstream.rs | 24 +- server/src/hottier.rs | 625 ++++++++++++-------------- server/src/option.rs | 19 +- server/src/storage/localfs.rs | 8 +- server/src/storage/object_storage.rs | 6 +- server/src/storage/s3.rs | 27 +- server/src/utils.rs | 24 +- server/src/validator.rs | 57 +-- 9 files changed, 322 insertions(+), 503 deletions(-) diff --git a/server/src/cli.rs b/server/src/cli.rs index e451c1ab1..34c8f51a2 100644 --- a/server/src/cli.rs +++ b/server/src/cli.rs @@ -104,12 +104,6 @@ pub struct Cli { /// The local hot_tier path is used for optimising the query performance in the distributed systems pub hot_tier_storage_path: Option, - - /// Size for local cache - pub min_hot_tier_size: u64, - - ///maximum disk usage for hot tier - pub max_disk_usage: f64, } impl Cli { @@ -144,8 +138,6 @@ impl Cli { pub const FLIGHT_PORT: &'static str = "flight-port"; pub const CORS: &'static str = "cors"; pub const HOT_TIER_PATH: &'static str = "hot-tier-path"; - pub const MIN_HOT_TIER_SIZE: &'static str = "hot-tier-size"; - pub const MAX_DISK_USAGE: &'static str = "max-disk-usage"; pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) @@ -417,25 +409,6 @@ impl Cli { .help("Local path on this device to be used for hot tier data") .next_line_help(true), ) - .arg( - Arg::new(Self::MIN_HOT_TIER_SIZE) - .long(Self::MIN_HOT_TIER_SIZE) - .env("P_MIN_HOT_TIER_SIZE") - .value_name("size") - .default_value("100GiB") - .value_parser(validation::hot_tier_size) - .help("Minimum allowed cache size for all streams combined (In human readable format, e.g 1GiB, 2GiB, 100MB)") - .next_line_help(true), - ) - .arg( - Arg::new(Self::MAX_DISK_USAGE) - .long(Self::MAX_DISK_USAGE) - .env("P_MAX_DISK_USAGE") - .value_name("size") - .default_value("80.0") - .value_parser(validation::disk_usage) - .help("Maximum disk usage for hot tier"), - ) .group( ArgGroup::new("oidc") .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) @@ -574,15 +547,7 @@ impl FromArgMatches for Cli { }; self.hot_tier_storage_path = m.get_one::(Self::HOT_TIER_PATH).cloned(); - self.min_hot_tier_size = m - .get_one::(Self::MIN_HOT_TIER_SIZE) - .cloned() - .expect("default value for cache size"); - self.max_disk_usage = m - .get_one::(Self::MAX_DISK_USAGE) - .cloned() - .expect("default value for max disk usage"); Ok(()) } } diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 0d7b12fec..7aaf4fcfd 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -956,26 +956,15 @@ pub async fn put_stream_hot_tier( Err(err) => return Err(StreamError::InvalidHotTierConfig(err)), }; - validator::hot_tier( - &hottier.start_date, - &hottier.end_date, - &hottier.size.to_string(), - )?; + validator::hot_tier(&hottier.size.to_string())?; STREAM_INFO.set_hot_tier(&stream_name, true)?; if let Some(hot_tier_manager) = HotTierManager::global() { - hottier = StreamHotTier { - start_date: hottier.start_date, - end_date: hottier.end_date, - size: hottier.size.clone(), - used_size: Some("0GiB".to_string()), - available_size: Some(hottier.size), - updated_date_range: None, - }; - hot_tier_manager - .validate(&stream_name, &mut hottier) + .validate_hot_tier_size(&stream_name, &hottier.size) .await?; + hottier.used_size = Some("0GiB".to_string()); + hottier.available_size = Some(hottier.size.clone()); hot_tier_manager .put_hot_tier(&stream_name, &mut hottier) .await?; @@ -987,7 +976,10 @@ pub async fn put_stream_hot_tier( .await?; } - Ok((web::Json(hottier), StatusCode::OK)) + Ok(( + format!("hot tier set for stream {stream_name}"), + StatusCode::OK, + )) } pub async fn get_stream_hot_tier(req: HttpRequest) -> Result { diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 35bedb8de..b648bed45 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -17,6 +17,7 @@ */ use std::{ + collections::BTreeMap, io, path::{Path, PathBuf}, sync::Arc, @@ -27,15 +28,15 @@ use crate::{ metadata::{error::stream_info::MetadataError, STREAM_INFO}, option::{ validation::{bytes_to_human_size, human_size_to_bytes}, - CONFIG, + CONFIG, MAX_DISK_USAGE, }, storage::{ObjectStorage, ObjectStorageError}, - utils::{extract_datetime, get_dir_size}, - validator::{error::HotTierValidationError, parse_human_date}, + utils::extract_datetime, + validator::error::HotTierValidationError, }; -use chrono::{NaiveDate, Utc}; +use chrono::NaiveDate; use clokwerk::{AsyncScheduler, Interval, Job}; -use futures::TryStreamExt; +use futures::{stream::FuturesUnordered, StreamExt, TryStreamExt}; use futures_util::TryFutureExt; use object_store::{local::LocalFileSystem, ObjectStore}; use once_cell::sync::OnceCell; @@ -48,7 +49,7 @@ use tokio::io::AsyncWriteExt; use tokio_stream::wrappers::ReadDirStream; pub const STREAM_HOT_TIER_FILENAME: &str = ".hot_tier.json"; - +pub const MIN_STREAM_HOT_TIER_SIZE_BYTES: u64 = 10737418240; // 10 GiB const HOT_TIER_SYNC_DURATION: Interval = clokwerk::Interval::Minutes(1); #[derive(Debug, serde::Deserialize, serde::Serialize)] @@ -59,12 +60,8 @@ pub struct StreamHotTier { pub used_size: Option, #[serde(skip_serializing_if = "Option::is_none")] pub available_size: Option, - #[serde(rename = "start_date")] - pub start_date: String, - #[serde(rename = "end_date")] - pub end_date: String, #[serde(skip_serializing_if = "Option::is_none")] - pub updated_date_range: Option>, + pub oldest_date_time_entry: Option, } pub struct HotTierManager { @@ -88,83 +85,58 @@ impl HotTierManager { })) } - ///validate the hot tier configuration, check the disk availability - /// update the hot tier range if the disk is not available to download the entire set - /// delete the existing hot tier files once validation is successful - pub async fn validate( + ///get the total hot tier size for all streams + pub async fn get_hot_tiers_size(&self, current_stream: &str) -> Result { + let mut total_hot_tier_size = 0; + for stream in STREAM_INFO.list_streams() { + if self.check_stream_hot_tier_exists(&stream) && stream != current_stream { + let stream_hot_tier = self.get_hot_tier(&stream).await?; + total_hot_tier_size += human_size_to_bytes(&stream_hot_tier.size).unwrap(); + } + } + Ok(total_hot_tier_size) + } + + /// validate if hot tier size can be fit in the disk + /// check disk usage and hot tier size of all other streams + /// check if total hot tier size of all streams is less than max disk usage + /// delete all the files from hot tier once validation is successful and hot tier is ready to be updated + pub async fn validate_hot_tier_size( &self, stream: &str, - stream_hot_tier: &mut StreamHotTier, + size: &str, ) -> Result<(), HotTierError> { - async fn validate_helper( - this: &HotTierManager, - stream: &str, - stream_hot_tier: &mut StreamHotTier, - ) -> Result<(), HotTierError> { - let date_list = this.get_date_list( - &stream_hot_tier.start_date, - &stream_hot_tier.end_date, - &stream_hot_tier.updated_date_range, - )?; - let object_store = CONFIG.storage().get_object_store(); - let s3_file_list = object_store.list_files(stream).await?; - let mut manifest_list = Vec::new(); - let mut total_size_to_download = 0; - - for date in &date_list { - let date_str = date.to_string(); - let manifest_files_to_download = s3_file_list - .iter() - .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) - .collect::>(); - - for file in manifest_files_to_download { - let manifest_path: RelativePathBuf = RelativePathBuf::from(file); - let manifest_file = object_store.get_object(&manifest_path).await?; - let manifest: Manifest = serde_json::from_slice(&manifest_file)?; - manifest_list.push(manifest.clone()); - } - } + let (total_disk_space, available_disk_space, used_disk_space) = get_disk_usage(); - for manifest in &manifest_list { - total_size_to_download += manifest - .files - .iter() - .map(|file| file.file_size) - .sum::(); - } + if let (Some(total_disk_space), Some(available_disk_space), Some(used_disk_space)) = + (total_disk_space, available_disk_space, used_disk_space) + { + let stream_hot_tier_size = human_size_to_bytes(size).unwrap(); + let total_hot_tier_size = self.get_hot_tiers_size(stream).await?; + let projected_disk_usage = used_disk_space + total_hot_tier_size + stream_hot_tier_size; - if (!this - .is_disk_available(total_size_to_download, &stream_hot_tier.size) - .await? - || human_size_to_bytes(&stream_hot_tier.size).unwrap() < total_size_to_download) - && date_list.len() > 1 - { - stream_hot_tier.updated_date_range = Some( - date_list - .iter() - .skip(1) - .map(|date| date.to_string()) - .collect(), - ); - return Box::pin(validate_helper(this, stream, stream_hot_tier)).await; - } + let usage_percentage = (projected_disk_usage as f64 / total_disk_space as f64) * 100.0; - if let Ok(mut existing_hot_tier) = this.get_hot_tier(stream).await { - let available_date_list = this.fetch_hot_tier_dates(stream).await?; - this.delete_files_from_hot_tier( - &mut existing_hot_tier, - stream, - &available_date_list, - true, - ) - .await?; + if usage_percentage > MAX_DISK_USAGE { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( + "Including the hot tier size of all the streams, the projected disk usage will be {} which is above the set threshold of {}%, hence unable to set the hot tier for the stream. Total Disk Size: {}, Available Disk Size: {}, Used Disk Size: {}, Total Hot Tier Size (all other streams): {}", + usage_percentage, + MAX_DISK_USAGE, + bytes_to_human_size(total_disk_space), + bytes_to_human_size(available_disk_space), + bytes_to_human_size(used_disk_space), + bytes_to_human_size(total_hot_tier_size) + )))); } + } - Ok(()) + if self.check_stream_hot_tier_exists(stream) { + let available_date_list = self.fetch_hot_tier_dates(stream).await?; + self.delete_files_from_hot_tier(stream, &available_date_list) + .await?; } - validate_helper(self, stream, stream_hot_tier).await + Ok(()) } ///get the hot tier metadata file for the stream @@ -181,7 +153,12 @@ impl HotTierManager { .and_then(|resp| resp.bytes()) .await; match res { - Ok(bytes) => serde_json::from_slice(&bytes).map_err(Into::into), + Ok(bytes) => { + let mut stream_hot_tier: StreamHotTier = serde_json::from_slice(&bytes)?; + let oldest_date_time_entry = self.get_oldest_date_time_entry(stream).await?; + stream_hot_tier.oldest_date_time_entry = oldest_date_time_entry; + Ok(stream_hot_tier) + } Err(err) => Err(err.into()), } } @@ -193,21 +170,8 @@ impl HotTierManager { stream: &str, hot_tier: &mut StreamHotTier, ) -> Result<(), HotTierError> { - let date_list = if let Some(updated_date_range) = &hot_tier.updated_date_range { - updated_date_range - .iter() - .map(|date| NaiveDate::parse_from_str(date, "%Y-%m-%d").unwrap()) - .collect() - } else { - self.get_date_list(&hot_tier.start_date, &hot_tier.end_date, &None)? - }; - hot_tier.updated_date_range = date_list - .iter() - .map(|date| date.to_string()) - .collect::>() - .into(); let path = hot_tier_file_path(&self.hot_tier_path, stream)?; - let bytes = serde_json::to_vec(hot_tier)?.into(); + let bytes = serde_json::to_vec(&hot_tier)?.into(); self.filesystem.put(&path, bytes).await?; Ok(()) } @@ -239,9 +203,19 @@ impl HotTierManager { ///sync the hot tier files from S3 to the hot tier directory for all streams async fn sync_hot_tier(&self) -> Result<(), HotTierError> { let streams = STREAM_INFO.list_streams(); + let sync_hot_tier_tasks = FuturesUnordered::new(); for stream in streams { if self.check_stream_hot_tier_exists(&stream) { - self.process_stream(stream).await?; + sync_hot_tier_tasks.push(async move { self.process_stream(stream).await }); + //self.process_stream(stream).await?; + } + } + + let res: Vec<_> = sync_hot_tier_tasks.collect().await; + for res in res { + if let Err(err) = res { + log::error!("Failed to run hot tier sync task {err:?}"); + return Err(err); } } Ok(()) @@ -250,126 +224,77 @@ impl HotTierManager { ///process the hot tier files for the stream /// delete the files from the hot tier directory if the available date range is outside the hot tier range async fn process_stream(&self, stream: String) -> Result<(), HotTierError> { - let mut stream_hot_tier = self.get_hot_tier(&stream).await?; + let stream_hot_tier = self.get_hot_tier(&stream).await?; let mut parquet_file_size = human_size_to_bytes(stream_hot_tier.used_size.as_ref().unwrap()).unwrap(); - let date_list = self.get_hot_tier_time_range(&stream_hot_tier).await?; - let available_date_list = self.fetch_hot_tier_dates(&stream).await?; - let dates_to_delete: Vec = available_date_list - .into_iter() - .filter(|available_date| !date_list.contains(available_date)) - .collect(); - if !dates_to_delete.is_empty() { - self.delete_files_from_hot_tier(&mut stream_hot_tier, &stream, &dates_to_delete, false) - .await?; - } - let object_store = CONFIG.storage().get_object_store(); - let s3_file_list = object_store.list_files(&stream).await?; - for date in date_list { - self.process_date( - &stream, - &mut stream_hot_tier, - &s3_file_list, - date, - &mut parquet_file_size, - object_store.clone(), - ) - .await?; - self.put_hot_tier(&stream, &mut stream_hot_tier).await?; - } + let object_store = CONFIG.storage().get_object_store(); + let mut s3_manifest_file_list = object_store.list_manifest_files(&stream).await?; + self.process_manifest( + &stream, + &mut s3_manifest_file_list, + &mut parquet_file_size, + object_store.clone(), + ) + .await?; Ok(()) } - ///get the list of date range from hot tier metadata - /// if updated_date_range is available, get the list of dates from the updated_date_range - /// else get the list of dates from the start and end date - fn get_date_list( - &self, - start_date: &str, - end_date: &str, - updated_date_range: &Option>, - ) -> Result, HotTierError> { - let (dt_start_date, dt_end_date) = if let Some(updated_date_range) = updated_date_range { - parse_human_date( - updated_date_range.first().unwrap(), - updated_date_range.last().unwrap(), - )? - } else { - parse_human_date(start_date, end_date)? - }; - - let date_list: Vec = (0..) - .map(|i| dt_start_date + chrono::Duration::days(i)) - .take_while(|&date| date <= dt_end_date) - .collect(); - - Ok(date_list) - } - ///process the hot tier files for the date for the stream /// collect all manifests from S3 for the date, sort the parquet file list /// in order to download the latest files first /// download the parquet files if not present in hot tier directory - async fn process_date( + async fn process_manifest( &self, stream: &str, - stream_hot_tier: &mut StreamHotTier, - s3_file_list: &[String], - date: NaiveDate, + manifest_files_to_download: &mut BTreeMap>, parquet_file_size: &mut u64, object_store: Arc, ) -> Result<(), HotTierError> { - let date_str = date.to_string(); - let available_date_list = self.fetch_hot_tier_dates(stream).await?; - if available_date_list.contains(&date) && !date.eq(&Utc::now().date_naive()) { - return Ok(()); - } - let manifest_files_to_download = s3_file_list - .iter() - .filter(|file| file.starts_with(&format!("{}/date={}", stream, date_str))) - .collect::>(); if manifest_files_to_download.is_empty() { - self.update_hot_tier_time_range(stream, stream_hot_tier, &date_str) - .await?; - } - let mut manifest_list = Vec::new(); - for file in manifest_files_to_download { - let path = self.hot_tier_path.join(file); - fs::create_dir_all(path.parent().unwrap()).await?; - let manifest_path: RelativePathBuf = RelativePathBuf::from(file); - let manifest_file = object_store.get_object(&manifest_path).await?; - - let mut file = fs::File::create(path.clone()).await?; - file.write_all(&manifest_file).await?; - let manifest: Manifest = serde_json::from_slice(&manifest_file)?; - manifest_list.push(manifest.clone()); - } - let mut parquet_file_list = Vec::new(); - for manifest in manifest_list { - parquet_file_list.extend(manifest.files); + return Ok(()); } - parquet_file_list.sort_by_key(|file| file.file_path.clone()); - parquet_file_list.reverse(); - for parquet_file in parquet_file_list { - let parquet_file_path = &parquet_file.file_path; - let parquet_path = self.hot_tier_path.join(parquet_file_path); - if !parquet_path.exists() { - if !self - .process_parquet_file( - stream, - stream_hot_tier, - &parquet_file, - parquet_file_size, - parquet_path, - ) - .await? - { - break; + for (str_date, manifest_files) in manifest_files_to_download.iter().rev() { + let mut storage_combined_manifest = Manifest::default(); + + for manifest_file in manifest_files { + let manifest_path: RelativePathBuf = RelativePathBuf::from(manifest_file.clone()); + let storage_manifest_bytes = object_store.get_object(&manifest_path).await?; + let storage_manifest: Manifest = serde_json::from_slice(&storage_manifest_bytes)?; + storage_combined_manifest + .files + .extend(storage_manifest.files); + } + + storage_combined_manifest + .files + .sort_by_key(|file| file.file_path.clone()); + + while let Some(parquet_file) = storage_combined_manifest.files.pop() { + let parquet_file_path = &parquet_file.file_path; + let parquet_path = self.hot_tier_path.join(parquet_file_path); + + if !parquet_path.exists() { + if let Ok(date) = + NaiveDate::parse_from_str(str_date.trim_start_matches("date="), "%Y-%m-%d") + { + if !self + .process_parquet_file( + stream, + &parquet_file, + parquet_file_size, + parquet_path, + date, + ) + .await? + { + break; + } + } else { + log::warn!("Invalid date format: {}", str_date); + } } - } else { - continue; } } @@ -385,12 +310,13 @@ impl HotTierManager { async fn process_parquet_file( &self, stream: &str, - stream_hot_tier: &mut StreamHotTier, parquet_file: &File, parquet_file_size: &mut u64, parquet_path: PathBuf, + date: NaiveDate, ) -> Result { let mut file_processed = false; + let mut stream_hot_tier = self.get_hot_tier(stream).await?; if !self .is_disk_available(parquet_file.file_size, &stream_hot_tier.size) .await? @@ -398,9 +324,9 @@ impl HotTierManager { <= parquet_file.file_size { if !self - .delete_1minute_data( + .cleanup_hot_tier_old_data( stream, - stream_hot_tier, + &mut stream_hot_tier, &parquet_path, parquet_file.file_size, ) @@ -427,7 +353,19 @@ impl HotTierManager { human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() - parquet_file.file_size, )); + self.put_hot_tier(stream, &mut stream_hot_tier).await?; file_processed = true; + let mut hot_tier_manifest = self + .get_stream_hot_tier_manifest_for_date(stream, &date) + .await?; + hot_tier_manifest.files.push(parquet_file.clone()); + // write the manifest file to the hot tier directory + let manifest_path = self + .hot_tier_path + .join(stream) + .join(format!("date={}/hottier.manifest.json", date)); + fs::create_dir_all(manifest_path.parent().unwrap()).await?; + fs::write(manifest_path, serde_json::to_vec(&hot_tier_manifest)?).await?; Ok(file_processed) } @@ -435,60 +373,19 @@ impl HotTierManager { /// update the used and available size in the hot tier metadata pub async fn delete_files_from_hot_tier( &self, - stream_hot_tier: &mut StreamHotTier, stream: &str, dates: &[NaiveDate], - validate: bool, ) -> Result<(), HotTierError> { for date in dates.iter() { let path = self.hot_tier_path.join(format!("{}/date={}", stream, date)); if path.exists() { - if !validate { - let size = get_dir_size(path.clone())?; - if self - .check_current_date_for_deletion(stream_hot_tier, stream) - .await? - { - return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom( - format!( - "Hot tier capacity for stream {} is exhausted (Total: {}, Available - {}). Today's data cannot be deleted. Please increase the hot tier size. Download will resume tomorrow" - , stream, stream_hot_tier.size, stream_hot_tier.available_size.clone().unwrap()) ))); - } - stream_hot_tier.used_size = Some(bytes_to_human_size( - human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()).unwrap() - - size, - )); - stream_hot_tier.available_size = Some(bytes_to_human_size( - human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()) - .unwrap() - + size, - )); - self.put_hot_tier(stream, stream_hot_tier).await?; - } fs::remove_dir_all(path.clone()).await?; } - self.update_hot_tier_time_range(stream, stream_hot_tier, &date.to_string()) - .await?; } Ok(()) } - ///check if the current date is the last date available in the hot tier directory - async fn check_current_date_for_deletion( - &self, - stream_hot_tier: &StreamHotTier, - stream: &str, - ) -> Result { - let current_date = Utc::now().date_naive(); - let (_, end_date) = - parse_human_date(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; - let is_end_date_today = end_date == current_date; - let available_date_list = self.fetch_hot_tier_dates(stream).await?; - let is_current_date_available = available_date_list.contains(¤t_date); - Ok(available_date_list.len() == 1 && is_current_date_available && is_end_date_today) - } - ///fetch the list of dates available in the hot tier directory for the stream and sort them pub async fn fetch_hot_tier_dates(&self, stream: &str) -> Result, HotTierError> { let mut date_list = Vec::new(); @@ -511,62 +408,34 @@ impl HotTierManager { Ok(date_list) } - ///update the hot tier time range for the stream - /// remove the date given from the updated_date_range - /// put the hot tier metadata file for the stream - pub async fn update_hot_tier_time_range( + ///get hot tier manifest for the stream and date + pub async fn get_stream_hot_tier_manifest_for_date( &self, stream: &str, - stream_hot_tier: &mut StreamHotTier, - date: &str, - ) -> Result<(), HotTierError> { - if stream_hot_tier.updated_date_range.as_ref().unwrap().len() == 1 - && stream_hot_tier - .updated_date_range - .as_ref() - .unwrap() - .contains(&Utc::now().date_naive().to_string()) - { - return Ok(()); + date: &NaiveDate, + ) -> Result { + let mut hot_tier_manifest = Manifest::default(); + let path = self + .hot_tier_path + .join(stream) + .join(format!("date={}", date)); + if path.exists() { + let date_dirs = ReadDirStream::new(fs::read_dir(&path).await?); + let manifest_files: Vec = date_dirs.try_collect().await?; + for manifest in manifest_files { + if !manifest + .file_name() + .to_string_lossy() + .ends_with(".manifest.json") + { + continue; + } + let file = fs::read(manifest.path()).await?; + let manifest: Manifest = serde_json::from_slice(&file)?; + hot_tier_manifest.files.extend(manifest.files); + } } - let mut existing_date_range = stream_hot_tier.updated_date_range.as_ref().unwrap().clone(); - existing_date_range.retain(|d| d != date); - stream_hot_tier.updated_date_range = Some(existing_date_range); - - self.put_hot_tier(stream, stream_hot_tier).await?; - Ok(()) - } - - ///get the updated hot tier time range for the stream - /// get the existing dates from the updated_date_range - /// update the updated_date_range with the existing dates and the new dates - pub async fn get_hot_tier_time_range( - &self, - stream_hot_tier: &StreamHotTier, - ) -> Result, HotTierError> { - let (start_date, end_date) = - parse_human_date(&stream_hot_tier.start_date, &stream_hot_tier.end_date)?; - let date_list: Vec = (0..) - .map(|i| start_date + chrono::Duration::days(i)) - .take_while(|&date| date <= end_date) - .collect(); - let mut existing_date_range: Vec = stream_hot_tier - .updated_date_range - .as_ref() - .unwrap() - .iter() - .map(|date| NaiveDate::parse_from_str(date, "%Y-%m-%d").unwrap()) - .collect(); - existing_date_range.sort(); - let mut updated_date_range = vec![*date_list.last().unwrap()]; - updated_date_range.extend( - date_list - .into_iter() - .filter(|date| existing_date_range.contains(date)), - ); - updated_date_range.sort(); - updated_date_range.dedup(); - Ok(updated_date_range) + Ok(hot_tier_manifest) } ///get the list of files from all the manifests present in hot tier directory for the stream @@ -600,31 +469,12 @@ impl HotTierManager { let mut hot_tier_parquet_files: Vec = Vec::new(); let date_list = self.fetch_hot_tier_dates(stream).await?; for date in date_list { - let date_str = date.to_string(); - let path = &self - .hot_tier_path - .join(stream) - .join(format!("date={}", date_str)); - if !path.exists() { - continue; - } - - let date_dirs = ReadDirStream::new(fs::read_dir(&path).await?); - let manifest_files: Vec = date_dirs.try_collect().await?; - for manifest in manifest_files { - if !manifest - .file_name() - .to_string_lossy() - .ends_with(".manifest.json") - { - continue; - } - let file = fs::read(manifest.path()).await?; - let manifest: Manifest = serde_json::from_slice(&file)?; + let manifest = self + .get_stream_hot_tier_manifest_for_date(stream, &date) + .await?; - for parquet_file in manifest.files { - hot_tier_parquet_files.push(parquet_file.clone()); - } + for parquet_file in manifest.files { + hot_tier_parquet_files.push(parquet_file.clone()); } } Ok(hot_tier_parquet_files) @@ -645,7 +495,7 @@ impl HotTierManager { /// check for the oldest entry to delete if the path exists in hot tier /// update the used and available size in the hot tier metadata /// loop if available size is still less than the parquet file size - pub async fn delete_1minute_data( + pub async fn cleanup_hot_tier_old_data( &self, stream: &str, stream_hot_tier: &mut StreamHotTier, @@ -653,31 +503,26 @@ impl HotTierManager { parquet_file_size: u64, ) -> Result { let mut delete_successful = false; - let date_list = self.get_date_list( - &stream_hot_tier.start_date, - &stream_hot_tier.end_date, - &stream_hot_tier.updated_date_range, - )?; - - if let Some(date_to_delete) = date_list.first() { - let date_str = date_to_delete.to_string(); - let path = self + let dates = self.fetch_hot_tier_dates(stream).await?; + 'loop_dates: for date in dates { + let date_str = date.to_string(); + let path = &self .hot_tier_path .join(stream) .join(format!("date={}", date_str)); - if !path.exists() { - return Ok(delete_successful); + continue; } let date_dirs = ReadDirStream::new(fs::read_dir(&path).await?); let mut manifest_files: Vec = date_dirs.try_collect().await?; - manifest_files.retain(|file| { - file.file_name() + manifest_files.retain(|manifest| { + manifest + .file_name() .to_string_lossy() - .ends_with("manifest.json") + .ends_with(".manifest.json") }); - 'loop_manifests: for manifest_file in manifest_files { + for manifest_file in manifest_files { let file = fs::read(manifest_file.path()).await?; let mut manifest: Manifest = serde_json::from_slice(&file)?; @@ -705,7 +550,9 @@ impl HotTierManager { } fs::write(manifest_file.path(), serde_json::to_vec(&manifest)?).await?; - fs::remove_dir_all(&path_to_delete.parent().unwrap()).await?; + + fs::remove_dir_all(path_to_delete.parent().unwrap()).await?; + delete_empty_directory_hot_tier(path_to_delete.parent().unwrap()).await?; stream_hot_tier.used_size = Some(bytes_to_human_size( human_size_to_bytes(&stream_hot_tier.used_size.clone().unwrap()) @@ -726,7 +573,7 @@ impl HotTierManager { { continue 'loop_files; } else { - break 'loop_manifests; + break 'loop_dates; } } else { fs::write(manifest_file.path(), serde_json::to_vec(&manifest)?).await?; @@ -734,6 +581,7 @@ impl HotTierManager { } } } + Ok(delete_successful) } @@ -746,33 +594,32 @@ impl HotTierManager { ) -> Result { let (total_disk_space, available_disk_space, used_disk_space) = get_disk_usage(); - if let Some(available_space) = available_disk_space { + if let (Some(total_disk_space), Some(available_disk_space), Some(used_disk_space)) = + (total_disk_space, available_disk_space, used_disk_space) + { let hot_tier_size = human_size_to_bytes(hot_tier_size).unwrap(); - if available_space < hot_tier_size { + if available_disk_space < hot_tier_size { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( "Not enough space left in the disk for hot tier. Disk available Size: {}, Hot Tier Size: {}", - bytes_to_human_size(available_space), + bytes_to_human_size(available_disk_space), bytes_to_human_size(hot_tier_size) )))); } - if used_disk_space.unwrap() as f64 * 100.0 / total_disk_space.unwrap() as f64 - > CONFIG.parseable.max_disk_usage - { + if used_disk_space as f64 * 100.0 / total_disk_space as f64 > MAX_DISK_USAGE { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( "Disk usage is above the threshold. Disk Used Size: {}, Disk Total Size: {}, Disk Usage {}%", - bytes_to_human_size(used_disk_space.unwrap()), - bytes_to_human_size(total_disk_space.unwrap()), - used_disk_space.unwrap() as f64 * 100.0 / total_disk_space.unwrap() as f64 + bytes_to_human_size(used_disk_space), + bytes_to_human_size(total_disk_space), + used_disk_space as f64 * 100.0 / total_disk_space as f64 )))); } - if available_space < size_to_download { + if available_disk_space < size_to_download { return Ok(false); } - if ((used_disk_space.unwrap() + size_to_download) as f64 * 100.0 - / total_disk_space.unwrap() as f64) - > CONFIG.parseable.max_disk_usage + if ((used_disk_space + size_to_download) as f64 * 100.0 / total_disk_space as f64) + > MAX_DISK_USAGE { return Ok(false); } @@ -780,6 +627,58 @@ impl HotTierManager { Ok(true) } + + pub async fn get_oldest_date_time_entry( + &self, + stream: &str, + ) -> Result, HotTierError> { + let date_list = self.fetch_hot_tier_dates(stream).await?; + if date_list.is_empty() { + return Ok(None); + } + + for date in date_list { + let path = self + .hot_tier_path + .join(stream) + .join(format!("date={}", date)); + + let hours_dir = ReadDirStream::new(fs::read_dir(&path).await?); + let mut hours: Vec = hours_dir.try_collect().await?; + hours.retain(|entry| { + entry.path().is_dir() && entry.file_name().to_string_lossy().starts_with("hour=") + }); + hours.sort_by_key(|entry| entry.file_name().to_string_lossy().to_string()); + + for hour in hours { + let hour_str = hour + .file_name() + .to_string_lossy() + .trim_start_matches("hour=") + .to_string(); + + let minutes_dir = ReadDirStream::new(fs::read_dir(hour.path()).await?); + let mut minutes: Vec = minutes_dir.try_collect().await?; + minutes.retain(|entry| { + entry.path().is_dir() + && entry.file_name().to_string_lossy().starts_with("minute=") + }); + minutes.sort_by_key(|entry| entry.file_name().to_string_lossy().to_string()); + + if let Some(minute) = minutes.first() { + let minute_str = minute + .file_name() + .to_string_lossy() + .trim_start_matches("minute=") + .to_string(); + let oldest_date_time = format!("{} {}:{}:00", date, hour_str, minute_str); + return Ok(Some(oldest_date_time)); + } + } + } + + Ok(None) + } } /// get the hot tier file path for the stream @@ -817,6 +716,38 @@ pub fn get_disk_usage() -> (Option, Option, Option) { (None, None, None) } +async fn delete_empty_directory_hot_tier(path: &Path) -> io::Result<()> { + async fn delete_helper(path: &Path) -> io::Result<()> { + if path.is_dir() { + let mut read_dir = fs::read_dir(path).await?; + let mut subdirs = vec![]; + + while let Some(entry) = read_dir.next_entry().await? { + let entry_path = entry.path(); + if entry_path.is_dir() { + subdirs.push(entry_path); + } + } + let mut tasks = vec![]; + for subdir in &subdirs { + tasks.push(delete_empty_directory_hot_tier(subdir)); + } + futures::stream::iter(tasks) + .buffer_unordered(10) + .try_collect::>() + .await?; + + // Re-check the directory after deleting its subdirectories + let mut read_dir = fs::read_dir(path).await?; + if read_dir.next_entry().await?.is_none() { + fs::remove_dir(path).await?; + } + } + Ok(()) + } + delete_helper(path).await +} + #[derive(Debug, thiserror::Error)] pub enum HotTierError { #[error("{0}")] diff --git a/server/src/option.rs b/server/src/option.rs index 07be94f90..6a4b2f218 100644 --- a/server/src/option.rs +++ b/server/src/option.rs @@ -29,6 +29,8 @@ use std::env; use std::path::PathBuf; use std::sync::Arc; pub const MIN_CACHE_SIZE_BYTES: u64 = 1073741824; // 1 GiB + +pub const MAX_DISK_USAGE: f64 = 80.0; //max disk usage is 80% pub const JOIN_COMMUNITY: &str = "Join us on Parseable Slack community for questions : https://logg.ing/community"; pub static CONFIG: Lazy> = Lazy::new(|| Arc::new(Config::new())); @@ -377,21 +379,4 @@ pub mod validation { } Ok(size) } - - pub fn hot_tier_size(s: &str) -> Result { - let size = human_size_to_bytes(s)?; - if size < MIN_CACHE_SIZE_BYTES { - return Err( - "Specified value of cache size is smaller than current minimum of 1GiB".to_string(), - ); - } - Ok(size) - } - - pub fn disk_usage(s: &str) -> Result { - let disk_usage = s - .parse::() - .map_err(|_| "Invalid disk usage value".to_string())?; - Ok(disk_usage) - } } diff --git a/server/src/storage/localfs.rs b/server/src/storage/localfs.rs index d914e2693..4aa2d65d8 100644 --- a/server/src/storage/localfs.rs +++ b/server/src/storage/localfs.rs @@ -17,6 +17,7 @@ */ use std::{ + collections::BTreeMap, path::{Path, PathBuf}, sync::Arc, time::Instant, @@ -411,9 +412,12 @@ impl ObjectStorage for LocalFS { Ok(dates.into_iter().flatten().collect()) } - async fn list_files(&self, _stream_name: &str) -> Result, ObjectStorageError> { + async fn list_manifest_files( + &self, + _stream_name: &str, + ) -> Result>, ObjectStorageError> { //unimplemented - Ok(vec![]) + Ok(BTreeMap::new()) } async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError> { diff --git a/server/src/storage/object_storage.rs b/server/src/storage/object_storage.rs index e8e045f23..67119a164 100644 --- a/server/src/storage/object_storage.rs +++ b/server/src/storage/object_storage.rs @@ -49,6 +49,7 @@ use itertools::Itertools; use relative_path::RelativePath; use relative_path::RelativePathBuf; +use std::collections::BTreeMap; use std::{ collections::HashMap, fs, @@ -87,7 +88,10 @@ pub trait ObjectStorage: Sync + 'static { async fn get_all_saved_filters(&self) -> Result, ObjectStorageError>; async fn get_all_dashboards(&self) -> Result, ObjectStorageError>; async fn list_dates(&self, stream_name: &str) -> Result, ObjectStorageError>; - async fn list_files(&self, stream_name: &str) -> Result, ObjectStorageError>; + async fn list_manifest_files( + &self, + stream_name: &str, + ) -> Result>, ObjectStorageError>; async fn upload_file(&self, key: &str, path: &Path) -> Result<(), ObjectStorageError>; async fn delete_object(&self, path: &RelativePath) -> Result<(), ObjectStorageError>; async fn get_ingestor_meta_file_paths( diff --git a/server/src/storage/s3.rs b/server/src/storage/s3.rs index d545fa354..ff4d26162 100644 --- a/server/src/storage/s3.rs +++ b/server/src/storage/s3.rs @@ -33,6 +33,7 @@ use relative_path::{RelativePath, RelativePathBuf}; use tokio::fs::OpenOptions; use tokio::io::{AsyncReadExt, AsyncWriteExt}; +use std::collections::BTreeMap; use std::iter::Iterator; use std::path::Path as StdPath; use std::sync::Arc; @@ -343,8 +344,11 @@ impl S3 { Ok(dates) } - async fn _list_files(&self, stream: &str) -> Result, ObjectStorageError> { - let mut result_file_list = vec![]; + async fn _list_manifest_files( + &self, + stream: &str, + ) -> Result>, ObjectStorageError> { + let mut result_file_list: BTreeMap> = BTreeMap::new(); let resp = self .client .list_with_delimiter(Some(&(stream.into()))) @@ -357,17 +361,17 @@ impl S3 { .filter(|name| name.as_ref() != stream && name.as_ref() != STREAM_ROOT_DIRECTORY) .map(|name| name.as_ref().to_string()) .collect::>(); - for date in dates.clone() { - let hour_path = object_store::path::Path::from(format!("{}/{}", stream, date)); - let resp = self.client.list_with_delimiter(Some(&hour_path)).await?; - let manifest: Vec = resp + for date in dates { + let date_path = object_store::path::Path::from(format!("{}/{}", stream, &date)); + let resp = self.client.list_with_delimiter(Some(&date_path)).await?; + let manifests: Vec = resp .objects .iter() + .filter(|name| name.location.filename().unwrap().ends_with("manifest.json")) .map(|name| name.location.to_string()) .collect(); - result_file_list.extend(manifest); + result_file_list.entry(date).or_default().extend(manifests); } - Ok(result_file_list) } async fn _upload_file(&self, key: &str, path: &StdPath) -> Result<(), ObjectStorageError> { @@ -631,8 +635,11 @@ impl ObjectStorage for S3 { Ok(streams) } - async fn list_files(&self, stream_name: &str) -> Result, ObjectStorageError> { - let files = self._list_files(stream_name).await?; + async fn list_manifest_files( + &self, + stream_name: &str, + ) -> Result>, ObjectStorageError> { + let files = self._list_manifest_files(stream_name).await?; Ok(files) } diff --git a/server/src/utils.rs b/server/src/utils.rs index b574bbfa8..34630c375 100644 --- a/server/src/utils.rs +++ b/server/src/utils.rs @@ -28,8 +28,7 @@ use itertools::Itertools; use regex::Regex; use sha2::{Digest, Sha256}; use std::collections::HashMap; -use std::path::PathBuf; -use std::{env, fs}; +use std::env; use url::Url; #[allow(dead_code)] pub fn hostname() -> Option { @@ -306,27 +305,6 @@ pub fn get_ingestor_id() -> String { result } -pub fn get_dir_size(path: PathBuf) -> Result { - fn dir_size_recursive(path: &PathBuf) -> Result { - let mut size = 0; - - for entry in fs::read_dir(path)? { - let entry = entry?; - let metadata = entry.metadata()?; - - if metadata.is_dir() { - size += dir_size_recursive(&entry.path())?; - } else { - size += metadata.len(); - } - } - - Ok(size) - } - - dir_size_recursive(&path) -} - pub fn extract_datetime(path: &str) -> Option { let re = Regex::new(r"date=(\d{4}-\d{2}-\d{2})/hour=(\d{2})/minute=(\d{2})").unwrap(); if let Some(caps) = re.captures(path) { diff --git a/server/src/validator.rs b/server/src/validator.rs index 471a1cd6c..cd8b69710 100644 --- a/server/src/validator.rs +++ b/server/src/validator.rs @@ -16,7 +16,6 @@ * */ -use chrono::{NaiveDate, Utc}; use error::HotTierValidationError; use self::error::{AlertValidationError, StreamNameValidationError, UsernameValidationError}; @@ -24,8 +23,8 @@ use crate::alerts::rule::base::{NumericRule, StringRule}; use crate::alerts::rule::{ColumnRule, ConsecutiveNumericRule, ConsecutiveStringRule}; use crate::alerts::{Alerts, Rule}; use crate::handlers::http::cluster::INTERNAL_STREAM_NAME; +use crate::hottier::MIN_STREAM_HOT_TIER_SIZE_BYTES; use crate::option::validation::{bytes_to_human_size, human_size_to_bytes}; -use crate::option::CONFIG; // Add more sql keywords here in lower case const DENIED_NAMES: &[&str] = &[ @@ -149,56 +148,21 @@ pub fn user_name(username: &str) -> Result<(), UsernameValidationError> { Ok(()) } -pub fn hot_tier( - start_date: &str, - end_date: &str, - size: &str, -) -> Result<(), HotTierValidationError> { +pub fn hot_tier(size: &str) -> Result<(), HotTierValidationError> { if human_size_to_bytes(size).is_err() { return Err(HotTierValidationError::Size(bytes_to_human_size( - CONFIG.parseable.min_hot_tier_size, + MIN_STREAM_HOT_TIER_SIZE_BYTES, ))); } - if human_size_to_bytes(size).unwrap() < CONFIG.parseable.min_hot_tier_size { + if human_size_to_bytes(size).unwrap() < MIN_STREAM_HOT_TIER_SIZE_BYTES { return Err(HotTierValidationError::Size(bytes_to_human_size( - CONFIG.parseable.min_hot_tier_size, + MIN_STREAM_HOT_TIER_SIZE_BYTES, ))); } - let (start_date, end_date) = parse_human_date(start_date, end_date)?; - - if start_date >= end_date { - return Err(HotTierValidationError::DateRange); - } - Ok(()) } - -pub fn parse_human_date( - start_time: &str, - end_time: &str, -) -> Result<(NaiveDate, NaiveDate), HotTierValidationError> { - let start: NaiveDate; - let end: NaiveDate; - - if end_time == "now" { - end = Utc::now().naive_utc().date(); - start = if let Ok(parsed_date) = NaiveDate::parse_from_str(start_time, "%Y-%m-%d") { - parsed_date - } else { - end - chrono::Duration::from_std(humantime::parse_duration(start_time)?) - .map_err(|_| HotTierValidationError::StartDate)? - }; - } else { - start = NaiveDate::parse_from_str(start_time, "%Y-%m-%d") - .map_err(|_| HotTierValidationError::StartDate)?; - end = NaiveDate::parse_from_str(end_time, "%Y-%m-%d") - .map_err(|_| HotTierValidationError::EndDate)?; - }; - - Ok((start, end)) -} pub mod error { #[derive(Debug, thiserror::Error)] @@ -251,17 +215,6 @@ pub mod error { pub enum HotTierValidationError { #[error("Invalid size given for hot tier, please provide size in human readable format, e.g 1GiB, 2GiB, minimum {0}")] Size(String), - #[error( - "Invalid start date given for hot tier, please provide the date in yyyy-mm-dd format" - )] - StartDate, - #[error( - "Invalid end date given for hot tier, please provide the date in yyyy-mm-dd format" - )] - EndDate, - - #[error("End date should be greater than start date")] - DateRange, #[error("While generating times for 'now' failed to parse duration")] NotValidDuration(#[from] humantime::DurationError), From 52f603f0600c8eba43c269b69b5e05c2ec09b1f6 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Thu, 1 Aug 2024 08:17:12 +0530 Subject: [PATCH 09/12] fix for disk availability check removed check for hot tier size while processing the files as this check is done when setting the hot tier --- server/src/hottier.rs | 27 ++------------------------- 1 file changed, 2 insertions(+), 25 deletions(-) diff --git a/server/src/hottier.rs b/server/src/hottier.rs index b648bed45..56ad41829 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -317,9 +317,7 @@ impl HotTierManager { ) -> Result { let mut file_processed = false; let mut stream_hot_tier = self.get_hot_tier(stream).await?; - if !self - .is_disk_available(parquet_file.file_size, &stream_hot_tier.size) - .await? + if !self.is_disk_available(parquet_file.file_size).await? || human_size_to_bytes(&stream_hot_tier.available_size.clone().unwrap()).unwrap() <= parquet_file.file_size { @@ -587,33 +585,12 @@ impl HotTierManager { ///check if the disk is available to download the parquet file /// check if the disk usage is above the threshold - pub async fn is_disk_available( - &self, - size_to_download: u64, - hot_tier_size: &str, - ) -> Result { + pub async fn is_disk_available(&self, size_to_download: u64) -> Result { let (total_disk_space, available_disk_space, used_disk_space) = get_disk_usage(); if let (Some(total_disk_space), Some(available_disk_space), Some(used_disk_space)) = (total_disk_space, available_disk_space, used_disk_space) { - let hot_tier_size = human_size_to_bytes(hot_tier_size).unwrap(); - if available_disk_space < hot_tier_size { - return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( - "Not enough space left in the disk for hot tier. Disk available Size: {}, Hot Tier Size: {}", - bytes_to_human_size(available_disk_space), - bytes_to_human_size(hot_tier_size) - )))); - } - - if used_disk_space as f64 * 100.0 / total_disk_space as f64 > MAX_DISK_USAGE { - return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( - "Disk usage is above the threshold. Disk Used Size: {}, Disk Total Size: {}, Disk Usage {}%", - bytes_to_human_size(used_disk_space), - bytes_to_human_size(total_disk_space), - used_disk_space as f64 * 100.0 / total_disk_space as f64 - )))); - } if available_disk_space < size_to_download { return Ok(false); } From df4cde7d9dc7f5150f1d74ba11288dbffec96d01 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Thu, 1 Aug 2024 11:25:16 +0530 Subject: [PATCH 10/12] enhancement to hot tier implementation allow increasing the size of the existing hot tier restrict from reducing the size --- server/src/hottier.rs | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 56ad41829..9ce87f4c9 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -106,6 +106,18 @@ impl HotTierManager { stream: &str, size: &str, ) -> Result<(), HotTierError> { + if self.check_stream_hot_tier_exists(stream) { + //delete existing hot tier if its size is less than the updated hot tier size else return error + let existing_hot_tier = self.get_hot_tier(stream).await?; + if human_size_to_bytes(size) < human_size_to_bytes(&existing_hot_tier.size) { + return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( + "The hot tier size for the stream is already set to {} which is greater than the updated hot tier size of {}, reducing the hot tier size is not allowed", + existing_hot_tier.size, + size + )))); + } + } + let (total_disk_space, available_disk_space, used_disk_space) = get_disk_usage(); if let (Some(total_disk_space), Some(available_disk_space), Some(used_disk_space)) = @@ -113,7 +125,7 @@ impl HotTierManager { { let stream_hot_tier_size = human_size_to_bytes(size).unwrap(); let total_hot_tier_size = self.get_hot_tiers_size(stream).await?; - let projected_disk_usage = used_disk_space + total_hot_tier_size + stream_hot_tier_size; + let projected_disk_usage = total_hot_tier_size + stream_hot_tier_size; let usage_percentage = (projected_disk_usage as f64 / total_disk_space as f64) * 100.0; @@ -130,12 +142,6 @@ impl HotTierManager { } } - if self.check_stream_hot_tier_exists(stream) { - let available_date_list = self.fetch_hot_tier_dates(stream).await?; - self.delete_files_from_hot_tier(stream, &available_date_list) - .await?; - } - Ok(()) } From 06d2f4a38fe8ea35e206183ed02b40db56482871 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Thu, 1 Aug 2024 15:42:32 +0530 Subject: [PATCH 11/12] enhancement of hot tier implementation - added api to delete the hot tier for a stream DELETE /logstream/{logstream}/hottier - exposed max disk usage to env var P_MAX_DISK_USAGE defaulted to 80.0 --- server/src/cli.rs | 18 +++++++++ server/src/handlers/http/logstream.rs | 27 ++++++++++++++ server/src/handlers/http/modal/server.rs | 5 +++ server/src/hottier.rs | 47 ++++++++++++++++++------ server/src/option.rs | 13 ++++++- server/src/rbac/role.rs | 3 ++ 6 files changed, 100 insertions(+), 13 deletions(-) diff --git a/server/src/cli.rs b/server/src/cli.rs index 34c8f51a2..f0c665fe0 100644 --- a/server/src/cli.rs +++ b/server/src/cli.rs @@ -104,6 +104,9 @@ pub struct Cli { /// The local hot_tier path is used for optimising the query performance in the distributed systems pub hot_tier_storage_path: Option, + + ///maximum disk usage allowed + pub max_disk_usage: f64, } impl Cli { @@ -138,6 +141,7 @@ impl Cli { pub const FLIGHT_PORT: &'static str = "flight-port"; pub const CORS: &'static str = "cors"; pub const HOT_TIER_PATH: &'static str = "hot-tier-path"; + pub const MAX_DISK_USAGE: &'static str = "max-disk-usage"; pub fn local_stream_data_path(&self, stream_name: &str) -> PathBuf { self.local_staging_path.join(stream_name) @@ -409,6 +413,16 @@ impl Cli { .help("Local path on this device to be used for hot tier data") .next_line_help(true), ) + .arg( + Arg::new(Self::MAX_DISK_USAGE) + .long(Self::MAX_DISK_USAGE) + .env("P_MAX_DISK_USAGE") + .value_name("percentage") + .default_value("80.0") + .value_parser(validation::validate_disk_usage) + .help("Maximum allowed disk usage in percentage e.g 90.0 for 90%") + .next_line_help(true), + ) .group( ArgGroup::new("oidc") .args([Self::OPENID_CLIENT_ID, Self::OPENID_CLIENT_SECRET, Self::OPENID_ISSUER]) @@ -547,6 +561,10 @@ impl FromArgMatches for Cli { }; self.hot_tier_storage_path = m.get_one::(Self::HOT_TIER_PATH).cloned(); + self.max_disk_usage = m + .get_one::(Self::MAX_DISK_USAGE) + .cloned() + .expect("default for max disk usage"); Ok(()) } diff --git a/server/src/handlers/http/logstream.rs b/server/src/handlers/http/logstream.rs index 7aaf4fcfd..1a777efdd 100644 --- a/server/src/handlers/http/logstream.rs +++ b/server/src/handlers/http/logstream.rs @@ -1010,6 +1010,33 @@ pub async fn get_stream_hot_tier(req: HttpRequest) -> Result Result { + if CONFIG.parseable.mode != Mode::Query { + return Err(StreamError::Custom { + msg: "Hot tier can only be enabled in query mode".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + + let stream_name: String = req.match_info().get("logstream").unwrap().parse().unwrap(); + + if !metadata::STREAM_INFO.stream_exists(&stream_name) { + return Err(StreamError::StreamNotFound(stream_name)); + } + + if CONFIG.parseable.hot_tier_storage_path.is_none() { + return Err(StreamError::HotTierNotEnabled(stream_name)); + } + + if let Some(hot_tier_manager) = HotTierManager::global() { + hot_tier_manager.delete_hot_tier(&stream_name).await?; + } + Ok(( + format!("hot tier deleted for stream {stream_name}"), + StatusCode::OK, + )) +} #[allow(unused)] fn classify_json_error(kind: serde_json::error::Category) -> StatusCode { match kind { diff --git a/server/src/handlers/http/modal/server.rs b/server/src/handlers/http/modal/server.rs index 1878edc47..213d279a4 100644 --- a/server/src/handlers/http/modal/server.rs +++ b/server/src/handlers/http/modal/server.rs @@ -356,6 +356,11 @@ impl Server { web::get() .to(logstream::get_stream_hot_tier) .authorize_for_stream(Action::GetHotTierEnabled), + ) + .route( + web::delete() + .to(logstream::delete_stream_hot_tier) + .authorize_for_stream(Action::DeleteHotTierEnabled), ), ), ) diff --git a/server/src/hottier.rs b/server/src/hottier.rs index 9ce87f4c9..df130ba5b 100644 --- a/server/src/hottier.rs +++ b/server/src/hottier.rs @@ -28,7 +28,7 @@ use crate::{ metadata::{error::stream_info::MetadataError, STREAM_INFO}, option::{ validation::{bytes_to_human_size, human_size_to_bytes}, - CONFIG, MAX_DISK_USAGE, + CONFIG, }, storage::{ObjectStorage, ObjectStorageError}, utils::extract_datetime, @@ -86,15 +86,21 @@ impl HotTierManager { } ///get the total hot tier size for all streams - pub async fn get_hot_tiers_size(&self, current_stream: &str) -> Result { + pub async fn get_hot_tiers_size( + &self, + current_stream: &str, + ) -> Result<(u64, u64), HotTierError> { let mut total_hot_tier_size = 0; + let mut total_hot_tier_used_size = 0; for stream in STREAM_INFO.list_streams() { if self.check_stream_hot_tier_exists(&stream) && stream != current_stream { let stream_hot_tier = self.get_hot_tier(&stream).await?; total_hot_tier_size += human_size_to_bytes(&stream_hot_tier.size).unwrap(); + total_hot_tier_used_size += + human_size_to_bytes(&stream_hot_tier.used_size.unwrap()).unwrap(); } } - Ok(total_hot_tier_size) + Ok((total_hot_tier_size, total_hot_tier_used_size)) } /// validate if hot tier size can be fit in the disk @@ -106,9 +112,12 @@ impl HotTierManager { stream: &str, size: &str, ) -> Result<(), HotTierError> { + let mut existing_hot_tier_used_size = 0; if self.check_stream_hot_tier_exists(stream) { //delete existing hot tier if its size is less than the updated hot tier size else return error let existing_hot_tier = self.get_hot_tier(stream).await?; + existing_hot_tier_used_size = + human_size_to_bytes(&existing_hot_tier.used_size.unwrap()).unwrap(); if human_size_to_bytes(size) < human_size_to_bytes(&existing_hot_tier.size) { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( "The hot tier size for the stream is already set to {} which is greater than the updated hot tier size of {}, reducing the hot tier size is not allowed", @@ -124,16 +133,18 @@ impl HotTierManager { (total_disk_space, available_disk_space, used_disk_space) { let stream_hot_tier_size = human_size_to_bytes(size).unwrap(); - let total_hot_tier_size = self.get_hot_tiers_size(stream).await?; - let projected_disk_usage = total_hot_tier_size + stream_hot_tier_size; - - let usage_percentage = (projected_disk_usage as f64 / total_disk_space as f64) * 100.0; - - if usage_percentage > MAX_DISK_USAGE { + let (total_hot_tier_size, total_hot_tier_used_size) = + self.get_hot_tiers_size(stream).await?; + let projected_disk_usage = total_hot_tier_size + stream_hot_tier_size + used_disk_space + - existing_hot_tier_used_size + - total_hot_tier_used_size; + let usage_percentage = + ((projected_disk_usage as f64 / total_disk_space as f64) * 100.0).round(); + if usage_percentage > CONFIG.parseable.max_disk_usage { return Err(HotTierError::ObjectStorageError(ObjectStorageError::Custom(format!( - "Including the hot tier size of all the streams, the projected disk usage will be {} which is above the set threshold of {}%, hence unable to set the hot tier for the stream. Total Disk Size: {}, Available Disk Size: {}, Used Disk Size: {}, Total Hot Tier Size (all other streams): {}", + "Including the hot tier size of all the streams, the projected disk usage will be {}% which is above the set threshold of {}%, hence unable to set the hot tier for the stream. Total Disk Size: {}, Available Disk Size: {}, Used Disk Size: {}, Total Hot Tier Size (all other streams): {}", usage_percentage, - MAX_DISK_USAGE, + CONFIG.parseable.max_disk_usage, bytes_to_human_size(total_disk_space), bytes_to_human_size(available_disk_space), bytes_to_human_size(used_disk_space), @@ -169,6 +180,18 @@ impl HotTierManager { } } + pub async fn delete_hot_tier(&self, stream: &str) -> Result<(), HotTierError> { + if self.check_stream_hot_tier_exists(stream) { + let path = self.hot_tier_path.join(stream); + fs::remove_dir_all(path).await?; + Ok(()) + } else { + Err(HotTierError::HotTierValidationError( + HotTierValidationError::NotFound(stream.to_owned()), + )) + } + } + ///put the hot tier metadata file for the stream /// set the updated_date_range in the hot tier metadata file pub async fn put_hot_tier( @@ -602,7 +625,7 @@ impl HotTierManager { } if ((used_disk_space + size_to_download) as f64 * 100.0 / total_disk_space as f64) - > MAX_DISK_USAGE + > CONFIG.parseable.max_disk_usage { return Ok(false); } diff --git a/server/src/option.rs b/server/src/option.rs index 6a4b2f218..fa90b2cd2 100644 --- a/server/src/option.rs +++ b/server/src/option.rs @@ -30,7 +30,6 @@ use std::path::PathBuf; use std::sync::Arc; pub const MIN_CACHE_SIZE_BYTES: u64 = 1073741824; // 1 GiB -pub const MAX_DISK_USAGE: f64 = 80.0; //max disk usage is 80% pub const JOIN_COMMUNITY: &str = "Join us on Parseable Slack community for questions : https://logg.ing/community"; pub static CONFIG: Lazy> = Lazy::new(|| Arc::new(Config::new())); @@ -379,4 +378,16 @@ pub mod validation { } Ok(size) } + + pub fn validate_disk_usage(max_disk_usage: &str) -> Result { + if let Ok(max_disk_usage) = max_disk_usage.parse::() { + if (0.0..=100.0).contains(&max_disk_usage) { + Ok(max_disk_usage) + } else { + Err("Invalid value for max disk usage. It should be between 0 and 100".to_string()) + } + } else { + Err("Invalid value for max disk usage. It should be given as 90.0 for 90%".to_string()) + } + } } diff --git a/server/src/rbac/role.rs b/server/src/rbac/role.rs index 0edc1bf7c..1532c7c21 100644 --- a/server/src/rbac/role.rs +++ b/server/src/rbac/role.rs @@ -34,6 +34,7 @@ pub enum Action { PutCacheEnabled, PutHotTierEnabled, GetHotTierEnabled, + DeleteHotTierEnabled, PutAlert, GetAlert, PutUser, @@ -121,6 +122,7 @@ impl RoleBuilder { | Action::Deleteingestor | Action::PutHotTierEnabled | Action::GetHotTierEnabled + | Action::DeleteHotTierEnabled | Action::ListDashboard | Action::GetDashboard | Action::CreateDashboard @@ -212,6 +214,7 @@ pub mod model { Action::GetCacheEnabled, Action::PutHotTierEnabled, Action::GetHotTierEnabled, + Action::DeleteHotTierEnabled, Action::PutAlert, Action::GetAlert, Action::GetAbout, From 6d2b54ed0d431a264dacdb38f5421bba77279db1 Mon Sep 17 00:00:00 2001 From: Nikhil Sinha Date: Sat, 3 Aug 2024 19:09:58 +0530 Subject: [PATCH 12/12] P_MAX_DISK_USAGE env var renamed to P_MAX_DISK_USAGE_PERCENT --- server/src/cli.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/cli.rs b/server/src/cli.rs index f0c665fe0..857c165d3 100644 --- a/server/src/cli.rs +++ b/server/src/cli.rs @@ -416,7 +416,7 @@ impl Cli { .arg( Arg::new(Self::MAX_DISK_USAGE) .long(Self::MAX_DISK_USAGE) - .env("P_MAX_DISK_USAGE") + .env("P_MAX_DISK_USAGE_PERCENT") .value_name("percentage") .default_value("80.0") .value_parser(validation::validate_disk_usage)