diff --git a/.gitignore b/.gitignore index 853182f54..a11e5f556 100644 --- a/.gitignore +++ b/.gitignore @@ -10,7 +10,7 @@ key.pem helm-releases/.DS_Store .DS_Store env-file -parseable +parseable/* parseable_* parseable-env-secret cache diff --git a/src/about.rs b/src/about.rs index 36b4afffc..01294e3a5 100644 --- a/src/about.rs +++ b/src/about.rs @@ -17,10 +17,6 @@ * */ -use crate::analytics; -use crate::option::Config; -use crate::storage::StorageMetadata; -use crate::utils::update::{self, LatestRelease}; use chrono::Duration; use chrono_humanize::{Accuracy, Tense}; use crossterm::style::Stylize; @@ -30,6 +26,11 @@ use std::path::Path; use sysinfo::System; use ulid::Ulid; +use crate::analytics; +use crate::cli::Options; +use crate::storage::StorageMetadata; +use crate::utils::update::{self, LatestRelease}; + // Expose some static variables for internal usage pub static LATEST_RELEASE: OnceCell> = OnceCell::new(); @@ -99,7 +100,7 @@ impl ParseableVersion { pub fn print_about( current_version: semver::Version, - latest_release: Option, + latest_release: Option, commit_hash: String, ) { eprint!( @@ -123,7 +124,7 @@ pub fn print_about( ); } -fn print_latest_release(latest_release: update::LatestRelease) { +fn print_latest_release(latest_release: LatestRelease) { let time_since_latest_release = chrono::Utc::now() - latest_release.date; let time_since_latest_release = humanize_time(time_since_latest_release); let fmt_latest_version = format!( @@ -133,10 +134,10 @@ fn print_latest_release(latest_release: update::LatestRelease) { eprint!("{}", fmt_latest_version.red()); } -pub async fn print(config: &Config, meta: &StorageMetadata) { +pub async fn print(options: &Options, meta: &StorageMetadata) { // print current version let current = current(); - let latest_release = if config.options.check_update { + let latest_release = if options.check_update { update::get_latest(&meta.deployment_id).await.ok() } else { None diff --git a/src/alerts/mod.rs b/src/alerts/mod.rs index 71b0aed2a..eaa58d3e3 100644 --- a/src/alerts/mod.rs +++ b/src/alerts/mod.rs @@ -36,7 +36,7 @@ use ulid::Ulid; pub mod alerts_utils; pub mod target; -use crate::option::CONFIG; +use crate::parseable::PARSEABLE; use crate::query::{TableScanVisitor, QUERY_SESSION}; use crate::rbac::map::SessionKey; use crate::storage; @@ -650,8 +650,8 @@ impl AlertConfig { fn get_context(&self) -> Context { let deployment_instance = format!( "{}://{}", - CONFIG.options.get_scheme(), - CONFIG.options.address + PARSEABLE.options.get_scheme(), + PARSEABLE.options.address ); let deployment_id = storage::StorageMetadata::global().deployment_id; let deployment_mode = storage::StorageMetadata::global().mode.to_string(); @@ -730,7 +730,7 @@ impl Alerts { /// Loads alerts from disk, blocks pub async fn load(&self) -> Result<(), AlertError> { let mut map = self.alerts.write().await; - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); for alert in store.get_alerts().await.unwrap_or_default() { let (outbox_tx, outbox_rx) = oneshot::channel::<()>(); @@ -792,7 +792,7 @@ impl Alerts { new_state: AlertState, trigger_notif: Option, ) -> Result<(), AlertError> { - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); // read and modify alert let mut alert = self.get_alert_by_id(alert_id).await?; diff --git a/src/analytics.rs b/src/analytics.rs index a5e65f99b..42af72d36 100644 --- a/src/analytics.rs +++ b/src/analytics.rs @@ -16,16 +16,6 @@ * * */ - -use crate::about::{current, platform}; -use crate::handlers::http::cluster::utils::check_liveness; -use crate::handlers::http::{base_path_without_preceding_slash, cluster}; -use crate::handlers::STREAM_NAME_HEADER_KEY; -use crate::option::{Mode, CONFIG}; -use crate::{metadata, stats}; -use crate::{storage, HTTP_CLIENT}; - -use crate::stats::Stats; use actix_web::{web, HttpRequest, Responder}; use chrono::{DateTime, Utc}; use clokwerk::{AsyncScheduler, Interval}; @@ -40,6 +30,21 @@ use sysinfo::System; use tracing::{error, info}; use ulid::Ulid; +use crate::{ + about::{current, platform}, + handlers::{ + http::{ + base_path_without_preceding_slash, + cluster::{self, utils::check_liveness}, + }, + STREAM_NAME_HEADER_KEY, + }, + option::Mode, + parseable::PARSEABLE, + stats::{self, Stats}, + storage, HTTP_CLIENT, +}; + const ANALYTICS_SERVER_URL: &str = "https://analytics.parseable.io:80"; const ANALYTICS_SEND_INTERVAL_SECONDS: Interval = clokwerk::Interval::Hours(1); @@ -111,8 +116,8 @@ impl Report { cpu_count, memory_total_bytes: mem_total, platform: platform().to_string(), - storage_mode: CONFIG.get_storage_mode_string().to_string(), - server_mode: CONFIG.options.mode, + storage_mode: PARSEABLE.get_storage_mode_string().to_string(), + server_mode: PARSEABLE.options.mode, version: current().released_version.to_string(), commit_hash: current().commit_hash, active_ingestors: ingestor_metrics.0, @@ -148,7 +153,7 @@ pub async fn get_analytics(_: HttpRequest) -> impl Responder { } fn total_streams() -> usize { - metadata::STREAM_INFO.list_streams().len() + PARSEABLE.streams.len() } fn total_event_stats() -> (Stats, Stats, Stats) { @@ -164,7 +169,7 @@ fn total_event_stats() -> (Stats, Stats, Stats) { let mut deleted_parquet_bytes: u64 = 0; let mut deleted_json_bytes: u64 = 0; - for stream in metadata::STREAM_INFO.list_streams() { + for stream in PARSEABLE.streams.list() { let Some(stats) = stats::get_current_stats(&stream, "json") else { continue; }; @@ -219,7 +224,7 @@ async fn fetch_ingestors_metrics( let mut vec = vec![]; let mut active_ingestors = 0u64; let mut offline_ingestors = 0u64; - if CONFIG.options.mode == Mode::Query { + if PARSEABLE.options.mode == Mode::Query { // send analytics for ingest servers // ingestor infos should be valid here, if not some thing is wrong diff --git a/src/audit.rs b/src/audit.rs index 766413bab..e62c53c0e 100644 --- a/src/audit.rs +++ b/src/audit.rs @@ -21,9 +21,8 @@ use std::{ fmt::{Debug, Display}, }; -use crate::{about::current, storage::StorageMetadata, HTTP_CLIENT}; +use crate::{about::current, parseable::PARSEABLE, storage::StorageMetadata, HTTP_CLIENT}; -use super::option::CONFIG; use chrono::{DateTime, Utc}; use once_cell::sync::Lazy; use serde::Serialize; @@ -47,7 +46,12 @@ impl AuditLogger { // Try to construct the log endpoint URL by joining the base URL // with the ingest path, This can fail if the URL is not valid, // when the base URL is not set or the ingest path is not valid - let log_endpoint = match CONFIG.options.audit_logger.as_ref()?.join("/api/v1/ingest") { + let log_endpoint = match PARSEABLE + .options + .audit_logger + .as_ref()? + .join("/api/v1/ingest") + { Ok(url) => url, Err(err) => { eprintln!("Couldn't setup audit logger: {err}"); @@ -66,8 +70,8 @@ impl AuditLogger { .header("x-p-stream", "audit_log"); // Use basic auth if credentials are configured - if let Some(username) = CONFIG.options.audit_username.as_ref() { - req = req.basic_auth(username, CONFIG.options.audit_password.as_ref()) + if let Some(username) = PARSEABLE.options.audit_username.as_ref() { + req = req.basic_auth(username, PARSEABLE.options.audit_password.as_ref()) } match req.send().await { diff --git a/src/banner.rs b/src/banner.rs index 00ebb7e81..97055587c 100644 --- a/src/banner.rs +++ b/src/banner.rs @@ -21,14 +21,14 @@ use crossterm::style::Stylize; use crate::about; use crate::utils::uid::Uid; -use crate::{option::Config, storage::StorageMetadata}; +use crate::{parseable::Parseable, storage::StorageMetadata}; -pub async fn print(config: &Config, meta: &StorageMetadata) { +pub async fn print(config: &Parseable, meta: &StorageMetadata) { print_ascii_art(); let scheme = config.options.get_scheme(); status_info(config, &scheme, meta.deployment_id); storage_info(config).await; - about::print(config, meta).await; + about::print(&config.options, meta).await; println!(); } @@ -46,7 +46,7 @@ fn print_ascii_art() { eprint!("{ascii_name}"); } -fn status_info(config: &Config, scheme: &str, id: Uid) { +fn status_info(config: &Parseable, scheme: &str, id: Uid) { let address = format!( "\"{}://{}\" ({}), \":{}\" (livetail), \":{}\" (flight protocol)", scheme, @@ -59,7 +59,7 @@ fn status_info(config: &Config, scheme: &str, id: Uid) { let mut credentials = String::from("\"As set in P_USERNAME and P_PASSWORD environment variables\""); - if config.is_default_creds() { + if config.options.is_default_creds() { credentials = "\"Using default creds admin, admin. Please set credentials with P_USERNAME and P_PASSWORD.\"".red().to_string(); } @@ -93,7 +93,7 @@ fn status_info(config: &Config, scheme: &str, id: Uid) { /// - Mode (`Local drive`, `S3 bucket`) /// - Staging (temporary landing point for incoming events) /// - Store (path where the data is stored and its latency) -async fn storage_info(config: &Config) { +async fn storage_info(config: &Parseable) { let storage = config.storage(); let latency = storage.get_object_store().get_latency().await; diff --git a/src/catalog/mod.rs b/src/catalog/mod.rs index 878568e34..633c0c5d3 100644 --- a/src/catalog/mod.rs +++ b/src/catalog/mod.rs @@ -18,30 +18,31 @@ use std::{io::ErrorKind, sync::Arc}; -use self::{column::Column, snapshot::ManifestItem}; -use crate::handlers; -use crate::handlers::http::base_path_without_preceding_slash; -use crate::metadata::STREAM_INFO; -use crate::metrics::{EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE_DATE, EVENTS_STORAGE_SIZE_DATE}; -use crate::option::{Mode, CONFIG}; -use crate::stats::{ - event_labels_date, get_current_stats, storage_size_labels_date, update_deleted_stats, -}; -use crate::{ - catalog::manifest::Manifest, - event::DEFAULT_TIMESTAMP_KEY, - query::PartialTimeFilter, - storage::{object_storage::manifest_path, ObjectStorage, ObjectStorageError}, -}; use chrono::{DateTime, Local, NaiveTime, Utc}; +use column::Column; +use manifest::Manifest; use relative_path::RelativePathBuf; +use snapshot::ManifestItem; use std::io::Error as IOError; use tracing::{error, info}; + +use crate::{ + event::DEFAULT_TIMESTAMP_KEY, + handlers::{self, http::base_path_without_preceding_slash}, + metrics::{EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE_DATE, EVENTS_STORAGE_SIZE_DATE}, + option::Mode, + parseable::PARSEABLE, + query::PartialTimeFilter, + stats::{event_labels_date, get_current_stats, storage_size_labels_date, update_deleted_stats}, + storage::{ + object_storage::manifest_path, ObjectStorage, ObjectStorageError, ObjectStoreFormat, + }, +}; +pub use manifest::create_from_parquet_file; + pub mod column; pub mod manifest; pub mod snapshot; -use crate::storage::ObjectStoreFormat; -pub use manifest::create_from_parquet_file; pub trait Snapshot { fn manifests(&self, time_predicates: &[PartialTimeFilter]) -> Vec; } @@ -263,7 +264,7 @@ async fn create_manifest( files: vec![change], ..Manifest::default() }; - let mut first_event_at = STREAM_INFO.get_first_event(stream_name)?; + let mut first_event_at = PARSEABLE.get_stream(stream_name)?.get_first_event(); if first_event_at.is_none() { if let Some(first_event) = manifest.files.first() { let time_partition = &meta.time_partition; @@ -279,13 +280,11 @@ async fn create_manifest( } }; first_event_at = Some(lower_bound.with_timezone(&Local).to_rfc3339()); - if let Err(err) = - STREAM_INFO.set_first_event_at(stream_name, first_event_at.as_ref().unwrap()) - { - error!( - "Failed to update first_event_at in streaminfo for stream {:?} {err:?}", - stream_name - ); + match PARSEABLE.get_stream(stream_name) { + Ok(stream) => stream.set_first_event_at(first_event_at.as_ref().unwrap()), + Err(err) => error!( + "Failed to update first_event_at in streaminfo for stream {stream_name:?}, error = {err:?}" + ), } } } @@ -332,11 +331,11 @@ pub async fn remove_manifest_from_snapshot( let manifests = &mut meta.snapshot.manifest_list; // Filter out items whose manifest_path contains any of the dates_to_delete manifests.retain(|item| !dates.iter().any(|date| item.manifest_path.contains(date))); - STREAM_INFO.reset_first_event_at(stream_name)?; + PARSEABLE.get_stream(stream_name)?.reset_first_event_at(); meta.first_event_at = None; storage.put_snapshot(stream_name, meta.snapshot).await?; } - match CONFIG.options.mode { + match PARSEABLE.options.mode { Mode::All | Mode::Ingest => { Ok(get_first_event(storage.clone(), stream_name, Vec::new()).await?) } @@ -350,10 +349,10 @@ pub async fn get_first_event( dates: Vec, ) -> Result, ObjectStorageError> { let mut first_event_at: String = String::default(); - match CONFIG.options.mode { + match PARSEABLE.options.mode { Mode::All | Mode::Ingest => { // get current snapshot - let stream_first_event = STREAM_INFO.get_first_event(stream_name)?; + let stream_first_event = PARSEABLE.get_stream(stream_name)?.get_first_event(); if stream_first_event.is_some() { first_event_at = stream_first_event.unwrap(); } else { @@ -393,7 +392,9 @@ pub async fn get_first_event( first_event_at = lower_bound.with_timezone(&Local).to_rfc3339(); meta.first_event_at = Some(first_event_at.clone()); storage.put_stream_manifest(stream_name, &meta).await?; - STREAM_INFO.set_first_event_at(stream_name, &first_event_at)?; + PARSEABLE + .get_stream(stream_name)? + .set_first_event_at(&first_event_at); } } } diff --git a/src/cli.rs b/src/cli.rs index a331cc581..46a79a36c 100644 --- a/src/cli.rs +++ b/src/cli.rs @@ -381,4 +381,8 @@ impl Options { origin, }) } + + pub fn is_default_creds(&self) -> bool { + self.username == DEFAULT_USERNAME && self.password == DEFAULT_PASSWORD + } } diff --git a/src/connectors/kafka/processor.rs b/src/connectors/kafka/processor.rs index e06835121..218af7b25 100644 --- a/src/connectors/kafka/processor.rs +++ b/src/connectors/kafka/processor.rs @@ -16,25 +16,28 @@ * */ -use crate::connectors::common::processor::Processor; -use crate::connectors::kafka::config::BufferConfig; -use crate::connectors::kafka::{ConsumerRecord, StreamConsumer, TopicPartition}; -use crate::event::format::EventFormat; -use crate::event::format::{self, LogSource}; -use crate::event::Event as ParseableEvent; -use crate::handlers::http::ingest::create_stream_if_not_exists; -use crate::metadata::STREAM_INFO; -use crate::storage::StreamType; +use std::{collections::HashMap, sync::Arc}; + use async_trait::async_trait; use chrono::Utc; use futures_util::StreamExt; use rdkafka::consumer::{CommitMode, Consumer}; use serde_json::Value; -use std::collections::HashMap; -use std::sync::Arc; use tokio_stream::wrappers::ReceiverStream; use tracing::{debug, error}; +use crate::{ + connectors::common::processor::Processor, + event::{ + format::{json, EventFormat, LogSource}, + Event as ParseableEvent, + }, + parseable::PARSEABLE, + storage::StreamType, +}; + +use super::{config::BufferConfig, ConsumerRecord, StreamConsumer, TopicPartition}; + #[derive(Default, Debug, Clone)] pub struct ParseableSinkProcessor; @@ -48,16 +51,19 @@ impl ParseableSinkProcessor { .map(|r| r.topic.as_str()) .unwrap_or_default(); - create_stream_if_not_exists(stream_name, StreamType::UserDefined, LogSource::Json).await?; + PARSEABLE + .create_stream_if_not_exists(stream_name, StreamType::UserDefined, LogSource::Json) + .await?; - let schema = STREAM_INFO.schema_raw(stream_name)?; - let time_partition = STREAM_INFO.get_time_partition(stream_name)?; - let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; - let schema_version = STREAM_INFO.get_schema_version(stream_name)?; + let stream = PARSEABLE.get_stream(stream_name)?; + let schema = stream.get_schema_raw(); + let time_partition = stream.get_time_partition(); + let static_schema_flag = stream.get_static_schema_flag(); + let schema_version = stream.get_schema_version(); let (json_vec, total_payload_size) = Self::json_vec(records); - let batch_json_event = format::json::Event { - data: Value::Array(json_vec.to_vec()), + let batch_json_event = json::Event { + data: Value::Array(json_vec), }; let (rb, is_first) = batch_json_event.into_recordbatch( diff --git a/src/connectors/mod.rs b/src/connectors/mod.rs index 757c49a97..d3d3dd73f 100644 --- a/src/connectors/mod.rs +++ b/src/connectors/mod.rs @@ -16,34 +16,32 @@ * */ -use crate::connectors::common::processor::Processor; -use crate::connectors::common::shutdown::Shutdown; -use crate::connectors::kafka::config::KafkaConfig; -use crate::connectors::kafka::consumer::KafkaStreams; -use crate::connectors::kafka::metrics::KafkaMetricsCollector; -use crate::connectors::kafka::processor::ParseableSinkProcessor; -use crate::connectors::kafka::rebalance_listener::RebalanceListener; -use crate::connectors::kafka::sink::KafkaSinkConnector; -use crate::connectors::kafka::state::StreamState; -use crate::connectors::kafka::{ConsumerRecord, KafkaContext}; -use crate::option::{Mode, CONFIG}; +use std::sync::Arc; + use actix_web_prometheus::PrometheusMetrics; +use common::{processor::Processor, shutdown::Shutdown}; +use kafka::{ + config::KafkaConfig, consumer::KafkaStreams, metrics::KafkaMetricsCollector, + processor::ParseableSinkProcessor, rebalance_listener::RebalanceListener, + sink::KafkaSinkConnector, state::StreamState, ConsumerRecord, KafkaContext, +}; use prometheus::Registry; -use std::sync::Arc; use tokio::sync::RwLock; use tracing::{info, warn}; +use crate::{option::Mode, parseable::PARSEABLE}; + pub mod common; pub mod kafka; pub async fn init(prometheus: &PrometheusMetrics) -> anyhow::Result<()> { - if matches!(CONFIG.options.mode, Mode::Ingest | Mode::All) { - match CONFIG.kafka_config.validate() { + if matches!(PARSEABLE.options.mode, Mode::Ingest | Mode::All) { + match PARSEABLE.kafka_config.validate() { Err(e) => { warn!("Kafka connector configuration invalid. {}", e); } Ok(_) => { - let config = CONFIG.kafka_config.clone(); + let config = PARSEABLE.kafka_config.clone(); let shutdown_handle = Shutdown::default(); let registry = prometheus.registry.clone(); let processor = ParseableSinkProcessor; diff --git a/src/correlation.rs b/src/correlation.rs index 22974ee95..150644902 100644 --- a/src/correlation.rs +++ b/src/correlation.rs @@ -35,7 +35,7 @@ use crate::{ rbac::RBACError, users::{CORRELATION_DIR, USERS_ROOT_DIR}, }, - option::CONFIG, + parseable::PARSEABLE, query::QUERY_SESSION, rbac::{map::SessionKey, Users}, storage::ObjectStorageError, @@ -53,7 +53,7 @@ pub struct Correlations(RwLock); impl Correlations { // Load correlations from storage pub async fn load(&self) -> anyhow::Result<()> { - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let all_correlations = store.get_all_correlations().await.unwrap_or_default(); for correlations_bytes in all_correlations.values().flatten() { @@ -122,8 +122,8 @@ impl Correlations { // Update in storage let correlation_bytes = serde_json::to_vec(&correlation)?.into(); let path = correlation.path(); - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() .put_object(&path, correlation_bytes) .await?; @@ -157,8 +157,8 @@ impl Correlations { // Update in storage let correlation_bytes = serde_json::to_vec(&updated_correlation)?.into(); let path = updated_correlation.path(); - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() .put_object(&path, correlation_bytes) .await?; @@ -190,8 +190,8 @@ impl Correlations { // Delete from storage let path = correlation.path(); - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() .delete_object(&path) .await?; diff --git a/src/event/mod.rs b/src/event/mod.rs index 1178c7138..8b599d4b8 100644 --- a/src/event/mod.rs +++ b/src/event/mod.rs @@ -25,7 +25,7 @@ use itertools::Itertools; use std::sync::Arc; use self::error::EventError; -use crate::{metadata, staging::STAGING, storage::StreamType}; +use crate::{metadata::update_stats, parseable::PARSEABLE, storage::StreamType, LOCK_EXPECT}; use chrono::NaiveDateTime; use std::collections::HashMap; @@ -63,7 +63,7 @@ impl Event { commit_schema(&self.stream_name, self.rb.schema())?; } - STAGING.get_or_create_stream(&self.stream_name).push( + PARSEABLE.get_or_create_stream(&self.stream_name).push( &key, &self.rb, self.parsed_timestamp, @@ -71,13 +71,13 @@ impl Event { self.stream_type, )?; - metadata::STREAM_INFO.update_stats( + update_stats( &self.stream_name, self.origin_format, self.origin_size, self.rb.num_rows(), self.parsed_timestamp, - )?; + ); crate::livetail::LIVETAIL.process(&self.stream_name, &self.rb); @@ -87,7 +87,7 @@ impl Event { pub fn process_unchecked(&self) -> Result<(), EventError> { let key = get_schema_key(&self.rb.schema().fields); - STAGING.get_or_create_stream(&self.stream_name).push( + PARSEABLE.get_or_create_stream(&self.stream_name).push( &key, &self.rb, self.parsed_timestamp, @@ -110,11 +110,14 @@ pub fn get_schema_key(fields: &[Arc]) -> String { } pub fn commit_schema(stream_name: &str, schema: Arc) -> Result<(), EventError> { - let mut stream_metadata = metadata::STREAM_INFO.write().expect("lock poisoned"); + let mut stream_metadata = PARSEABLE.streams.write().expect("lock poisoned"); let map = &mut stream_metadata .get_mut(stream_name) .expect("map has entry for this stream name") + .metadata + .write() + .expect(LOCK_EXPECT) .schema; let current_schema = Schema::new(map.values().cloned().collect::()); let schema = Schema::try_merge(vec![current_schema, schema.as_ref().clone()])?; @@ -126,16 +129,12 @@ pub fn commit_schema(stream_name: &str, schema: Arc) -> Result<(), Event pub mod error { use arrow_schema::ArrowError; - use crate::metadata::error::stream_info::MetadataError; - use crate::staging::StagingError; - use crate::storage::ObjectStorageError; + use crate::{parseable::StagingError, storage::ObjectStorageError}; #[derive(Debug, thiserror::Error)] pub enum EventError { #[error("Stream Writer Failed: {0}")] StreamWriter(#[from] StagingError), - #[error("Metadata Error: {0}")] - Metadata(#[from] MetadataError), #[error("Stream Writer Failed: {0}")] Arrow(#[from] ArrowError), #[error("ObjectStorage Error: {0}")] diff --git a/src/handlers/airplane.rs b/src/handlers/airplane.rs index 448c986ef..47f2acb1b 100644 --- a/src/handlers/airplane.rs +++ b/src/handlers/airplane.rs @@ -37,9 +37,8 @@ use crate::handlers::http::cluster::get_ingestor_info; use crate::handlers::http::query::{into_query, update_schema_when_distributed}; use crate::handlers::livetail::cross_origin_config; use crate::metrics::QUERY_EXECUTE_TIME; -use crate::option::CONFIG; +use crate::parseable::PARSEABLE; use crate::query::{TableScanVisitor, QUERY_SESSION}; -use crate::staging::STAGING; use crate::utils::arrow::flight::{ append_temporary_events, get_query_from_ticket, into_flight_data, run_do_get_rpc, send_to_ingester, @@ -56,7 +55,6 @@ use futures::stream; use tonic::{Request, Response, Status, Streaming}; use crate::handlers::livetail::extract_session_key; -use crate::metadata::STREAM_INFO; use crate::rbac; use crate::rbac::Users; @@ -113,9 +111,10 @@ impl FlightService for AirServiceImpl { let table_name = request.into_inner().path; let table_name = table_name[0].clone(); - let schema = STREAM_INFO - .schema(&table_name) - .map_err(|err| Status::failed_precondition(err.to_string()))?; + let schema = PARSEABLE + .get_stream(&table_name) + .map_err(|err| Status::failed_precondition(err.to_string()))? + .get_schema(); let options = IpcWriteOptions::default(); let schema_result = SchemaAsIpc::new(&schema, &options) @@ -237,7 +236,7 @@ impl FlightService for AirServiceImpl { if event.is_some() { // Clear staging of stream once airplane has taxied - STAGING.clear(&stream_name); + PARSEABLE.get_or_create_stream(&stream_name).clear(); } let time = time.elapsed().as_secs_f64(); @@ -287,13 +286,12 @@ impl FlightService for AirServiceImpl { } pub fn server() -> impl Future>> + Send { - let mut addr: SocketAddr = CONFIG + let mut addr: SocketAddr = PARSEABLE .options .address .parse() - .unwrap_or_else(|err| panic!("{}, failed to parse `{}` as a socket address. Please set the environment variable `P_ADDR` to `:` without the scheme (e.g., 192.168.1.1:8000). Please refer to the documentation: https://logg.ing/env for more details.", -CONFIG.options.address, err)); - addr.set_port(CONFIG.options.flight_port); + .unwrap_or_else(|err| panic!("{}, failed to parse `{}` as a socket address. Please set the environment variable `P_ADDR` to `:` without the scheme (e.g., 192.168.1.1:8000). Please refer to the documentation: https://logg.ing/env for more details.", PARSEABLE.options.address, err)); + addr.set_port(PARSEABLE.options.flight_port); let service = AirServiceImpl {}; @@ -305,7 +303,10 @@ CONFIG.options.address, err)); let cors = cross_origin_config(); - let identity = match (&CONFIG.options.tls_cert_path, &CONFIG.options.tls_key_path) { + let identity = match ( + &PARSEABLE.options.tls_cert_path, + &PARSEABLE.options.tls_key_path, + ) { (Some(cert), Some(key)) => { match (std::fs::read_to_string(cert), std::fs::read_to_string(key)) { (Ok(cert_file), Ok(key_file)) => { diff --git a/src/handlers/http/about.rs b/src/handlers/http/about.rs index 9e277b797..44d39e6b3 100644 --- a/src/handlers/http/about.rs +++ b/src/handlers/http/about.rs @@ -21,7 +21,8 @@ use serde_json::{json, Value}; use crate::{ about::{self, get_latest_release}, - option::{Mode, CONFIG}, + option::Mode, + parseable::PARSEABLE, storage::StorageMetadata, }; use std::path::PathBuf; @@ -41,7 +42,7 @@ use std::path::PathBuf; /// "staging": staging, /// "grpcPort": grpc_port, /// "store": { -/// "type": CONFIG.get_storage_mode_string(), +/// "type": PARSEABLE.get_storage_mode_string(), /// "path": store_endpoint /// } /// } @@ -61,31 +62,31 @@ pub async fn about() -> Json { let current_version = format!("v{}", current_release.released_version); let commit = current_release.commit_hash; let deployment_id = meta.deployment_id.to_string(); - let mode = CONFIG.get_server_mode_string(); - let staging = if CONFIG.options.mode == Mode::Query { + let mode = PARSEABLE.get_server_mode_string(); + let staging = if PARSEABLE.options.mode == Mode::Query { "".to_string() } else { - CONFIG.staging_dir().display().to_string() + PARSEABLE.staging_dir().display().to_string() }; - let grpc_port = CONFIG.options.grpc_port; + let grpc_port = PARSEABLE.options.grpc_port; - let store_endpoint = CONFIG.storage().get_endpoint(); - let is_llm_active = &CONFIG.options.open_ai_key.is_some(); + let store_endpoint = PARSEABLE.storage.get_endpoint(); + let is_llm_active = &PARSEABLE.options.open_ai_key.is_some(); let llm_provider = is_llm_active.then_some("OpenAI"); - let is_oidc_active = CONFIG.options.openid().is_some(); + let is_oidc_active = PARSEABLE.options.openid().is_some(); let ui_version = option_env!("UI_VERSION").unwrap_or("development"); - let hot_tier_details: String = if CONFIG.hot_tier_dir().is_none() { + let hot_tier_details: String = if PARSEABLE.hot_tier_dir().is_none() { "Disabled".to_string() } else { - let hot_tier_dir: &Option = CONFIG.hot_tier_dir(); + let hot_tier_dir: &Option = PARSEABLE.hot_tier_dir(); format!( "Enabled, Path: {}", hot_tier_dir.as_ref().unwrap().display(), ) }; - let ms_clarity_tag = &CONFIG.options.ms_clarity_tag; + let ms_clarity_tag = &PARSEABLE.options.ms_clarity_tag; Json(json!({ "version": current_version, @@ -103,7 +104,7 @@ pub async fn about() -> Json { "hotTier": hot_tier_details, "grpcPort": grpc_port, "store": { - "type": CONFIG.get_storage_mode_string(), + "type": PARSEABLE.get_storage_mode_string(), "path": store_endpoint }, "analytics": { diff --git a/src/handlers/http/alerts.rs b/src/handlers/http/alerts.rs index a619ae989..357cf4b87 100644 --- a/src/handlers/http/alerts.rs +++ b/src/handlers/http/alerts.rs @@ -17,7 +17,7 @@ */ use crate::{ - option::CONFIG, storage::object_storage::alert_json_path, sync::schedule_alert_task, + parseable::PARSEABLE, storage::object_storage::alert_json_path, sync::schedule_alert_task, utils::actix::extract_session_key_from_req, }; use actix_web::{ @@ -71,7 +71,7 @@ pub async fn post( let path = alert_json_path(alert.id); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let alert_bytes = serde_json::to_vec(&alert)?; store.put_object(&path, Bytes::from(alert_bytes)).await?; @@ -105,7 +105,7 @@ pub async fn delete(req: HttpRequest, alert_id: Path) -> Result; +use utils::{check_liveness, to_url_string, IngestionStats, QueriedStats, StorageStats}; -use self::utils::StorageStats; +use crate::handlers::http::ingest::ingest_internal_stream; +use crate::metrics::prom_utils::Metrics; +use crate::parseable::PARSEABLE; +use crate::rbac::role::model::DefaultPrivilege; +use crate::rbac::user::User; +use crate::stats::Stats; +use crate::storage::{ + ObjectStorageError, ObjectStoreFormat, PARSEABLE_ROOT_DIRECTORY, STREAM_ROOT_DIRECTORY, +}; +use crate::HTTP_CLIENT; use super::base_path_without_preceding_slash; +use super::ingest::PostError; +use super::logstream::error::StreamError; +use super::modal::IngestorMetadata; use super::rbac::RBACError; -use std::collections::HashSet; -use std::time::Duration; +use super::role::RoleError; + +type IngestorMetadataArr = Vec; -use super::modal::IngestorMetadata; -use clokwerk::{AsyncScheduler, Interval}; pub const INTERNAL_STREAM_NAME: &str = "pmeta"; const CLUSTER_METRICS_INTERVAL_SECONDS: Interval = clokwerk::Interval::Minutes(1); @@ -435,8 +434,8 @@ pub async fn fetch_stats_from_ingestors( stream_name: &str, ) -> Result, StreamError> { let path = RelativePathBuf::from_iter([stream_name, STREAM_ROOT_DIRECTORY]); - let obs = CONFIG - .storage() + let obs = PARSEABLE + .storage .get_object_store() .get_objects( Some(&path), @@ -634,7 +633,7 @@ pub async fn get_cluster_info() -> Result { &ingestor.domain_name, reachable, staging_path, - CONFIG.storage().get_endpoint(), + PARSEABLE.storage.get_endpoint(), error, status, )); @@ -654,7 +653,7 @@ pub async fn get_cluster_metrics() -> Result { // update the .query.json file and return the new ingestorMetadataArr pub async fn get_ingestor_info() -> anyhow::Result { - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let root_path = RelativePathBuf::from(PARSEABLE_ROOT_DIRECTORY); let arr = store @@ -679,7 +678,7 @@ pub async fn remove_ingestor(ingestor: Path) -> Result) -> Result> = Arc::new(Mutex::new(false)); } @@ -58,12 +61,12 @@ pub async fn shutdown() { *shutdown_flag = true; // Sync staging - STAGING.flush_all(); + PARSEABLE.flush_all_streams(); } pub async fn readiness() -> HttpResponse { // Check the object store connection - if CONFIG.storage().get_object_store().check().await.is_ok() { + if PARSEABLE.storage.get_object_store().check().await.is_ok() { HttpResponse::new(StatusCode::OK) } else { HttpResponse::new(StatusCode::SERVICE_UNAVAILABLE) diff --git a/src/handlers/http/ingest.rs b/src/handlers/http/ingest.rs index b3da07761..dab4f84df 100644 --- a/src/handlers/http/ingest.rs +++ b/src/handlers/http/ingest.rs @@ -16,31 +16,11 @@ * */ -use super::logstream::error::{CreateStreamError, StreamError}; -use super::modal::utils::ingest_utils::{flatten_and_push_logs, push_logs}; -use super::users::dashboards::DashboardError; -use super::users::filters::FiltersError; -use crate::event::format::LogSource; -use crate::event::{ - self, - error::EventError, - format::{self, EventFormat}, -}; -use crate::handlers::http::modal::utils::logstream_utils::create_stream_and_schema_from_storage; -use crate::handlers::{LOG_SOURCE_KEY, STREAM_NAME_HEADER_KEY}; -use crate::metadata::error::stream_info::MetadataError; -use crate::metadata::{SchemaVersion, STREAM_INFO}; -use crate::option::{Mode, CONFIG}; -use crate::otel::logs::flatten_otel_logs; -use crate::otel::metrics::flatten_otel_metrics; -use crate::otel::traces::flatten_otel_traces; -use crate::storage::{ObjectStorageError, StreamType}; -use crate::utils::header_parsing::ParseHeaderError; -use crate::utils::json::flatten::JsonFlattenError; +use std::collections::HashMap; + use actix_web::web::{Json, Path}; use actix_web::{http::header::ContentType, HttpRequest, HttpResponse}; use arrow_array::RecordBatch; -use arrow_schema::Schema; use bytes::Bytes; use chrono::Utc; use http::StatusCode; @@ -48,8 +28,25 @@ use opentelemetry_proto::tonic::logs::v1::LogsData; use opentelemetry_proto::tonic::metrics::v1::MetricsData; use opentelemetry_proto::tonic::trace::v1::TracesData; use serde_json::Value; -use std::collections::HashMap; -use std::sync::Arc; + +use crate::event::error::EventError; +use crate::event::format::{self, EventFormat, LogSource}; +use crate::handlers::{LOG_SOURCE_KEY, STREAM_NAME_HEADER_KEY}; +use crate::metadata::SchemaVersion; +use crate::option::Mode; +use crate::otel::logs::flatten_otel_logs; +use crate::otel::metrics::flatten_otel_metrics; +use crate::otel::traces::flatten_otel_traces; +use crate::parseable::{StreamNotFound, PARSEABLE}; +use crate::storage::{ObjectStorageError, StreamType}; +use crate::utils::header_parsing::ParseHeaderError; +use crate::utils::json::flatten::JsonFlattenError; +use crate::{event, LOCK_EXPECT}; + +use super::logstream::error::{CreateStreamError, StreamError}; +use super::modal::utils::ingest_utils::{flatten_and_push_logs, push_logs}; +use super::users::dashboards::DashboardError; +use super::users::filters::FiltersError; // Handler for POST /api/v1/ingest // ingests events by extracting stream name from header @@ -60,11 +57,12 @@ pub async fn ingest(req: HttpRequest, Json(json): Json) -> Result Result< let parsed_timestamp = Utc::now().naive_utc(); let (rb, is_first) = { let body_val: Value = serde_json::from_slice(&body)?; - let hash_map = STREAM_INFO.read().unwrap(); + let hash_map = PARSEABLE.streams.read().unwrap(); let schema = hash_map .get(&stream_name) - .ok_or(PostError::StreamNotFound(stream_name.clone()))? + .ok_or_else(|| StreamNotFound(stream_name.clone()))? + .metadata + .read() + .expect(LOCK_EXPECT) .schema .clone(); let event = format::json::Event { data: body_val }; @@ -128,7 +129,9 @@ pub async fn handle_otel_logs_ingestion( } let stream_name = stream_name.to_str().unwrap().to_owned(); - create_stream_if_not_exists(&stream_name, StreamType::UserDefined, LogSource::OtelLogs).await?; + PARSEABLE + .create_stream_if_not_exists(&stream_name, StreamType::UserDefined, LogSource::OtelLogs) + .await?; //custom flattening required for otel logs let logs: LogsData = serde_json::from_value(json)?; @@ -157,12 +160,13 @@ pub async fn handle_otel_metrics_ingestion( return Err(PostError::IncorrectLogSource(LogSource::OtelMetrics)); } let stream_name = stream_name.to_str().unwrap().to_owned(); - create_stream_if_not_exists( - &stream_name, - StreamType::UserDefined, - LogSource::OtelMetrics, - ) - .await?; + PARSEABLE + .create_stream_if_not_exists( + &stream_name, + StreamType::UserDefined, + LogSource::OtelMetrics, + ) + .await?; //custom flattening required for otel metrics let metrics: MetricsData = serde_json::from_value(json)?; @@ -192,7 +196,8 @@ pub async fn handle_otel_traces_ingestion( return Err(PostError::IncorrectLogSource(LogSource::OtelTraces)); } let stream_name = stream_name.to_str().unwrap().to_owned(); - create_stream_if_not_exists(&stream_name, StreamType::UserDefined, LogSource::OtelTraces) + PARSEABLE + .create_stream_if_not_exists(&stream_name, StreamType::UserDefined, LogSource::OtelTraces) .await?; //custom flattening required for otel traces @@ -213,21 +218,25 @@ pub async fn post_event( Json(json): Json, ) -> Result { let stream_name = stream_name.into_inner(); - let internal_stream_names = STREAM_INFO.list_internal_streams(); + + let internal_stream_names = PARSEABLE.streams.list_internal_streams(); if internal_stream_names.contains(&stream_name) { return Err(PostError::InternalStream(stream_name)); } - if !STREAM_INFO.stream_exists(&stream_name) { + if !PARSEABLE.streams.contains(&stream_name) { // For distributed deployments, if the stream not found in memory map, //check if it exists in the storage //create stream and schema from storage - if CONFIG.options.mode != Mode::All { - match create_stream_and_schema_from_storage(&stream_name).await { + if PARSEABLE.options.mode != Mode::All { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { Ok(true) => {} - Ok(false) | Err(_) => return Err(PostError::StreamNotFound(stream_name.clone())), + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } else { - return Err(PostError::StreamNotFound(stream_name.clone())); + return Err(StreamNotFound(stream_name.clone()).into()); } } @@ -261,46 +270,10 @@ pub async fn push_logs_unchecked( Ok(unchecked_event) } -// Check if the stream exists and create a new stream if doesn't exist -pub async fn create_stream_if_not_exists( - stream_name: &str, - stream_type: StreamType, - log_source: LogSource, -) -> Result { - let mut stream_exists = false; - if STREAM_INFO.stream_exists(stream_name) { - stream_exists = true; - return Ok(stream_exists); - } - - // For distributed deployments, if the stream not found in memory map, - //check if it exists in the storage - //create stream and schema from storage - if CONFIG.options.mode != Mode::All - && create_stream_and_schema_from_storage(stream_name).await? - { - return Ok(stream_exists); - } - - super::logstream::create_stream( - stream_name.to_string(), - "", - None, - "", - false, - Arc::new(Schema::empty()), - stream_type, - log_source, - ) - .await?; - - Ok(stream_exists) -} - #[derive(Debug, thiserror::Error)] pub enum PostError { - #[error("Stream {0} not found")] - StreamNotFound(String), + #[error("{0}")] + StreamNotFound(#[from] StreamNotFound), #[error("Could not deserialize into JSON object, {0}")] SerdeError(#[from] serde_json::Error), #[error("Header Error: {0}")] @@ -311,8 +284,6 @@ pub enum PostError { Invalid(#[from] anyhow::Error), #[error("{0}")] CreateStream(#[from] CreateStreamError), - #[error("Error: {0}")] - MetadataStreamError(#[from] MetadataError), #[allow(unused)] #[error("Error: {0}")] CustomError(String), @@ -353,7 +324,6 @@ impl actix_web::ResponseError for PostError { StatusCode::BAD_REQUEST } PostError::CreateStream(_) => StatusCode::INTERNAL_SERVER_ERROR, - PostError::MetadataStreamError(_) => StatusCode::INTERNAL_SERVER_ERROR, PostError::StreamNotFound(_) => StatusCode::NOT_FOUND, PostError::CustomError(_) => StatusCode::INTERNAL_SERVER_ERROR, PostError::NetworkError(_) => StatusCode::INTERNAL_SERVER_ERROR, diff --git a/src/handlers/http/llm.rs b/src/handlers/http/llm.rs index 611220e88..a19761939 100644 --- a/src/handlers/http/llm.rs +++ b/src/handlers/http/llm.rs @@ -22,10 +22,7 @@ use itertools::Itertools; use reqwest; use serde_json::{json, Value}; -use crate::{ - metadata::{error::stream_info::MetadataError, STREAM_INFO}, - option::CONFIG, -}; +use crate::{parseable::StreamNotFound, parseable::PARSEABLE}; const OPEN_AI_URL: &str = "https://api.openai.com/v1/chat/completions"; @@ -87,13 +84,13 @@ fn build_request_body(ai_prompt: String) -> impl serde::Serialize { } pub async fn make_llm_request(body: web::Json) -> Result { - let api_key = match &CONFIG.options.open_ai_key { + let api_key = match &PARSEABLE.options.open_ai_key { Some(api_key) if api_key.len() > 3 => api_key, _ => return Err(LLMError::InvalidAPIKey), }; let stream_name = &body.stream; - let schema = STREAM_INFO.schema(stream_name)?; + let schema = PARSEABLE.get_stream(stream_name)?.get_schema(); let filtered_schema = schema .flattened_fields() .into_iter() @@ -146,7 +143,7 @@ pub enum LLMError { #[error("{0}")] APIError(String), #[error("{0}")] - StreamDoesNotExist(#[from] MetadataError), + StreamDoesNotExist(#[from] StreamNotFound), } impl actix_web::ResponseError for LLMError { diff --git a/src/handlers/http/logstream.rs b/src/handlers/http/logstream.rs index 753bdde87..f71739d8c 100644 --- a/src/handlers/http/logstream.rs +++ b/src/handlers/http/logstream.rs @@ -16,58 +16,47 @@ * */ -use self::error::{CreateStreamError, StreamError}; +use self::error::StreamError; use super::cluster::utils::{merge_quried_stats, IngestionStats, QueriedStats, StorageStats}; -use super::cluster::{sync_streams_with_ingestors, INTERNAL_STREAM_NAME}; -use super::ingest::create_stream_if_not_exists; -use super::modal::utils::logstream_utils::{ - create_stream_and_schema_from_storage, create_update_stream, update_first_event_at, -}; use super::query::update_schema_when_distributed; -use crate::event::format::{override_data_type, LogSource}; -use crate::handlers::STREAM_TYPE_KEY; +use crate::event::format::override_data_type; use crate::hottier::{HotTierManager, StreamHotTier, CURRENT_HOT_TIER_VERSION}; -use crate::metadata::{SchemaVersion, STREAM_INFO}; +use crate::metadata::SchemaVersion; use crate::metrics::{EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE_DATE, EVENTS_STORAGE_SIZE_DATE}; -use crate::option::{Mode, CONFIG}; +use crate::option::Mode; +use crate::parseable::{StreamNotFound, PARSEABLE}; use crate::rbac::role::Action; use crate::rbac::Users; -use crate::staging::{Stream, STAGING}; -use crate::stats; use crate::stats::{event_labels_date, storage_size_labels_date, Stats}; use crate::storage::retention::Retention; use crate::storage::{StreamInfo, StreamType}; use crate::utils::actix::extract_session_key_from_req; +use crate::{stats, validator, LOCK_EXPECT}; -use crate::{metadata, validator}; -use actix_web::http::header::{self, HeaderMap}; use actix_web::http::StatusCode; use actix_web::web::{Json, Path}; use actix_web::{web, HttpRequest, Responder}; use arrow_json::reader::infer_json_schema_from_iterator; -use arrow_schema::{Field, Schema}; use bytes::Bytes; use chrono::Utc; -use http::{HeaderName, HeaderValue}; use itertools::Itertools; use serde_json::{json, Value}; -use std::collections::HashMap; use std::fs; -use std::num::NonZeroU32; -use std::str::FromStr; use std::sync::Arc; use tracing::warn; pub async fn delete(stream_name: Path) -> Result { let stream_name = stream_name.into_inner(); - if !metadata::STREAM_INFO.stream_exists(&stream_name) { - return Err(StreamError::StreamNotFound(stream_name)); + if !PARSEABLE.streams.contains(&stream_name) { + return Err(StreamNotFound(stream_name).into()); } - let objectstore = CONFIG.storage().get_object_store(); + let objectstore = PARSEABLE.storage.get_object_store(); + // Delete from storage objectstore.delete_stream(&stream_name).await?; - let stream_dir = Stream::new(&CONFIG.options, &stream_name); + // Delete from staging + let stream_dir = PARSEABLE.get_or_create_stream(&stream_name); if fs::remove_dir_all(&stream_dir.data_path).is_err() { warn!( "failed to delete local data for stream {}. Clean {} manually", @@ -82,8 +71,8 @@ pub async fn delete(stream_name: Path) -> Result Result { .map_err(|err| StreamError::Anyhow(anyhow::Error::msg(err.to_string())))?; // list all streams from storage - let res = CONFIG - .storage() + let res = PARSEABLE + .storage .get_object_store() .list_streams() .await @@ -134,18 +123,19 @@ pub async fn detect_schema(Json(json): Json) -> Result) -> Result { let stream_name = stream_name.into_inner(); - match STREAM_INFO.schema(&stream_name) { - Ok(_) => {} - Err(_) if CONFIG.options.mode == Mode::Query => { - if !create_stream_and_schema_from_storage(&stream_name).await? { - return Err(StreamError::StreamNotFound(stream_name.clone())); - } - } - Err(err) => return Err(StreamError::from(err)), - }; + // Ensure parseable is aware of stream in distributed mode + if PARSEABLE.options.mode == Mode::Query + && PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await? + { + return Err(StreamNotFound(stream_name.clone()).into()); + } + + let stream = PARSEABLE.get_stream(&stream_name)?; match update_schema_when_distributed(&vec![stream_name.clone()]).await { Ok(_) => { - let schema = STREAM_INFO.schema(&stream_name)?; + let schema = stream.get_schema(); Ok((web::Json(schema), StatusCode::OK)) } Err(err) => Err(StreamError::Custom { @@ -162,38 +152,33 @@ pub async fn put_stream( ) -> Result { let stream_name = stream_name.into_inner(); - create_update_stream(req.headers(), &body, &stream_name).await?; + PARSEABLE + .create_update_stream(req.headers(), &body, &stream_name) + .await?; Ok(("Log stream created", StatusCode::OK)) } pub async fn get_retention(stream_name: Path) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { - // For query mode, if the stream not found in memory map, - //check if it exists in the storage - //create stream and schema from storage - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { - Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), - } - } else { - return Err(StreamError::StreamNotFound(stream_name)); - } - } - let retention = STREAM_INFO.get_retention(&stream_name); - - match retention { - Ok(retention) => { - if let Some(retention) = retention { - Ok((web::Json(retention), StatusCode::OK)) - } else { - Ok((web::Json(Retention::default()), StatusCode::OK)) - } + // For query mode, if the stream not found in memory map, + //check if it exists in the storage + //create stream and schema from storage + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { + Ok(true) => {} + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } - Err(err) => Err(StreamError::from(err)), } + + let retention = PARSEABLE + .get_stream(&stream_name)? + .get_retention() + .unwrap_or_default(); + Ok((web::Json(retention), StatusCode::OK)) } pub async fn put_retention( @@ -202,34 +187,32 @@ pub async fn put_retention( ) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { - // For query mode, if the stream not found in memory map, - //check if it exists in the storage - //create stream and schema from storage - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { - Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), - } - } else { - return Err(StreamError::StreamNotFound(stream_name)); + // For query mode, if the stream not found in memory map, + //check if it exists in the storage + //create stream and schema from storage + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { + Ok(true) => {} + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } + let stream = PARSEABLE.get_stream(&stream_name)?; let retention: Retention = match serde_json::from_value(json) { Ok(retention) => retention, Err(err) => return Err(StreamError::InvalidRetentionConfig(err)), }; - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() .put_retention(&stream_name, &retention) .await?; - metadata::STREAM_INFO - .set_retention(&stream_name, retention) - .expect("retention set on existing stream"); + stream.set_retention(retention); Ok(( format!("set retention configuration for log stream {stream_name}"), @@ -267,17 +250,20 @@ pub async fn get_stats( ) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { + if !PARSEABLE.streams.contains(&stream_name) { // For query mode, if the stream not found in memory map, //check if it exists in the storage //create stream and schema from storage - if cfg!(not(test)) && CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } else { - return Err(StreamError::StreamNotFound(stream_name)); + return Err(StreamNotFound(stream_name).into()); } } @@ -301,14 +287,17 @@ pub async fn get_stats( } let stats = stats::get_current_stats(&stream_name, "json") - .ok_or(StreamError::StreamNotFound(stream_name.clone()))?; + .ok_or_else(|| StreamNotFound(stream_name.clone()))?; let ingestor_stats: Option> = None; - let hash_map = STREAM_INFO.read().expect("Readable"); + let hash_map = PARSEABLE.streams.read().expect("Readable"); let stream_meta = &hash_map .get(&stream_name) - .ok_or(StreamError::StreamNotFound(stream_name.clone()))?; + .ok_or_else(|| StreamNotFound(stream_name.clone()))? + .metadata + .read() + .expect(LOCK_EXPECT); let time = Utc::now(); @@ -365,98 +354,44 @@ pub async fn get_stats( Ok((web::Json(stats), StatusCode::OK)) } -#[allow(clippy::too_many_arguments)] -pub async fn create_stream( - stream_name: String, - time_partition: &str, - time_partition_limit: Option, - custom_partition: &str, - static_schema_flag: bool, - schema: Arc, - stream_type: StreamType, - log_source: LogSource, -) -> Result<(), CreateStreamError> { - // fail to proceed if invalid stream name - if stream_type != StreamType::Internal { - validator::stream_name(&stream_name, stream_type)?; - } - // Proceed to create log stream if it doesn't exist - let storage = CONFIG.storage().get_object_store(); - - match storage - .create_stream( - &stream_name, - time_partition, - time_partition_limit, - custom_partition, - static_schema_flag, - schema.clone(), - stream_type, - log_source.clone(), - ) - .await - { - Ok(created_at) => { - let mut static_schema: HashMap> = HashMap::new(); - - for (field_name, field) in schema - .fields() - .iter() - .map(|field| (field.name().to_string(), field.clone())) - { - static_schema.insert(field_name, field); - } - - metadata::STREAM_INFO.add_stream( - stream_name.to_string(), - created_at, - time_partition.to_string(), - time_partition_limit, - custom_partition.to_string(), - static_schema_flag, - static_schema, - stream_type, - SchemaVersion::V1, // New stream - log_source, - ); - } - Err(err) => { - return Err(CreateStreamError::Storage { stream_name, err }); - } - } - Ok(()) -} - pub async fn get_stream_info(stream_name: Path) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { + if !PARSEABLE.streams.contains(&stream_name) { + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } else { - return Err(StreamError::StreamNotFound(stream_name)); + return Err(StreamNotFound(stream_name).into()); } } - let storage = CONFIG.storage().get_object_store(); + let storage = PARSEABLE.storage.get_object_store(); // if first_event_at is not found in memory map, check if it exists in the storage // if it exists in the storage, update the first_event_at in memory map let stream_first_event_at = - if let Ok(Some(first_event_at)) = STREAM_INFO.get_first_event(&stream_name) { + if let Some(first_event_at) = PARSEABLE.get_stream(&stream_name)?.get_first_event() { Some(first_event_at) } else if let Ok(Some(first_event_at)) = storage.get_first_event_from_storage(&stream_name).await { - update_first_event_at(&stream_name, &first_event_at).await + PARSEABLE + .update_first_event_at(&stream_name, &first_event_at) + .await } else { None }; - let hash_map = STREAM_INFO.read().unwrap(); - let stream_meta = &hash_map + let hash_map = PARSEABLE.streams.read().unwrap(); + let stream_meta = hash_map .get(&stream_name) - .ok_or(StreamError::StreamNotFound(stream_name.clone()))?; + .ok_or_else(|| StreamNotFound(stream_name.clone()))? + .metadata + .read() + .expect(LOCK_EXPECT); let stream_info = StreamInfo { stream_type: stream_meta.stream_type, @@ -479,24 +414,22 @@ pub async fn put_stream_hot_tier( Json(mut hottier): Json, ) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { - // For query mode, if the stream not found in memory map, - //check if it exists in the storage - //create stream and schema from storage - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { - Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), - } - } else { - return Err(StreamError::StreamNotFound(stream_name)); + // For query mode, if the stream not found in memory map, + //check if it exists in the storage + //create stream and schema from storage + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { + Ok(true) => {} + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } - if STREAM_INFO - .stream_type(&stream_name) - .is_ok_and(|t| t == StreamType::Internal) - { + let stream = PARSEABLE.get_stream(&stream_name)?; + + if stream.get_stream_type() == StreamType::Internal { return Err(StreamError::Custom { msg: "Hot tier can not be updated for internal stream".to_string(), status: StatusCode::BAD_REQUEST, @@ -505,7 +438,7 @@ pub async fn put_stream_hot_tier( validator::hot_tier(&hottier.size.to_string())?; - STREAM_INFO.set_hot_tier(&stream_name, true)?; + stream.set_hot_tier(true); let Some(hot_tier_manager) = HotTierManager::global() else { return Err(StreamError::HotTierNotEnabled(stream_name)); }; @@ -518,7 +451,7 @@ pub async fn put_stream_hot_tier( hot_tier_manager .put_hot_tier(&stream_name, &mut hottier) .await?; - let storage = CONFIG.storage().get_object_store(); + let storage = PARSEABLE.storage.get_object_store(); let mut stream_metadata = storage.get_object_store_format(&stream_name).await?; stream_metadata.hot_tier_enabled = true; storage @@ -534,17 +467,20 @@ pub async fn put_stream_hot_tier( pub async fn get_stream_hot_tier(stream_name: Path) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { + if !PARSEABLE.streams.contains(&stream_name) { // For query mode, if the stream not found in memory map, //check if it exists in the storage //create stream and schema from storage - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } else { - return Err(StreamError::StreamNotFound(stream_name)); + return Err(StreamNotFound(stream_name).into()); } } @@ -561,34 +497,30 @@ pub async fn delete_stream_hot_tier( ) -> Result { let stream_name = stream_name.into_inner(); - if !STREAM_INFO.stream_exists(&stream_name) { - // For query mode, if the stream not found in memory map, - //check if it exists in the storage - //create stream and schema from storage - if CONFIG.options.mode == Mode::Query { - match create_stream_and_schema_from_storage(&stream_name).await { - Ok(true) => {} - Ok(false) | Err(_) => return Err(StreamError::StreamNotFound(stream_name.clone())), - } - } else { - return Err(StreamError::StreamNotFound(stream_name)); + // For query mode, if the stream not found in memory map, + //check if it exists in the storage + //create stream and schema from storage + if PARSEABLE.options.mode == Mode::Query { + match PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await + { + Ok(true) => {} + Ok(false) | Err(_) => return Err(StreamNotFound(stream_name.clone()).into()), } } - let Some(hot_tier_manager) = HotTierManager::global() else { - return Err(StreamError::HotTierNotEnabled(stream_name)); - }; - - if STREAM_INFO - .stream_type(&stream_name) - .is_ok_and(|t| t == StreamType::Internal) - { + if PARSEABLE.get_stream(&stream_name)?.get_stream_type() == StreamType::Internal { return Err(StreamError::Custom { msg: "Hot tier can not be deleted for internal stream".to_string(), status: StatusCode::BAD_REQUEST, }); } + let Some(hot_tier_manager) = HotTierManager::global() else { + return Err(StreamError::HotTierNotEnabled(stream_name)); + }; + hot_tier_manager.delete_hot_tier(&stream_name).await?; Ok(( @@ -597,27 +529,6 @@ pub async fn delete_stream_hot_tier( )) } -pub async fn create_internal_stream_if_not_exists() -> Result<(), StreamError> { - if let Ok(stream_exists) = - create_stream_if_not_exists(INTERNAL_STREAM_NAME, StreamType::Internal, LogSource::Pmeta) - .await - { - if stream_exists { - return Ok(()); - } - let mut header_map = HeaderMap::new(); - header_map.insert( - HeaderName::from_str(STREAM_TYPE_KEY).unwrap(), - HeaderValue::from_str(&StreamType::Internal.to_string()).unwrap(), - ); - header_map.insert( - header::CONTENT_TYPE, - HeaderValue::from_static("application/json"), - ); - sync_streams_with_ingestors(header_map, Bytes::new(), INTERNAL_STREAM_NAME).await?; - } - Ok(()) -} #[allow(unused)] fn classify_json_error(kind: serde_json::error::Category) -> StatusCode { match kind { @@ -635,7 +546,7 @@ pub mod error { use crate::{ hottier::HotTierError, - metadata::error::stream_info::MetadataError, + parseable::StreamNotFound, storage::ObjectStorageError, validator::error::{ AlertValidationError, HotTierValidationError, StreamNameValidationError, @@ -664,8 +575,8 @@ pub mod error { pub enum StreamError { #[error("{0}")] CreateStream(#[from] CreateStreamError), - #[error("Log stream {0} does not exist")] - StreamNotFound(String), + #[error("{0}")] + StreamNotFound(#[from] StreamNotFound), #[error("Log stream is not initialized, send an event to this logstream and try again")] UninitializedLogstream, #[error("Storage Error {0}")] @@ -747,28 +658,12 @@ pub mod error { .body(self.to_string()) } } - - impl From for StreamError { - fn from(value: MetadataError) -> Self { - match value { - MetadataError::StreamMetaNotFound(s) => StreamError::StreamNotFound(s), - MetadataError::StandaloneWithDistributed(s) => StreamError::Custom { - msg: s, - status: StatusCode::INTERNAL_SERVER_ERROR, - }, - } - } - } } #[cfg(test)] mod tests { - use crate::handlers::http::logstream::error::StreamError; - use crate::handlers::http::logstream::get_stats; use crate::handlers::http::modal::utils::logstream_utils::PutStreamHeaders; use actix_web::test::TestRequest; - use actix_web::web; - use anyhow::bail; // TODO: Fix this test with routes // #[actix_web::test] @@ -778,15 +673,15 @@ mod tests { // let _ = get_stats(req).await; // } - #[actix_web::test] - async fn get_stats_stream_not_found_error_for_unknown_logstream() -> anyhow::Result<()> { - let req = TestRequest::default().to_http_request(); + // #[actix_web::test] + // async fn get_stats_stream_not_found_error_for_unknown_logstream() -> anyhow::Result<()> { + // let req = TestRequest::default().to_http_request(); - match get_stats(req, web::Path::from("test".to_string())).await { - Err(StreamError::StreamNotFound(_)) => Ok(()), - _ => bail!("expected StreamNotFound error"), - } - } + // match get_stats(req, web::Path::from("test".to_string())).await { + // Err(StreamError::StreamNotFound(_)) => Ok(()), + // _ => bail!("expected StreamNotFound error"), + // } + // } #[actix_web::test] async fn header_without_log_source() { diff --git a/src/handlers/http/middleware.rs b/src/handlers/http/middleware.rs index 65fd82774..74d88e28c 100644 --- a/src/handlers/http/middleware.rs +++ b/src/handlers/http/middleware.rs @@ -33,9 +33,9 @@ use crate::{ STREAM_NAME_HEADER_KEY, }, option::Mode, + parseable::PARSEABLE, }; use crate::{ - option::CONFIG, rbac::Users, rbac::{self, role::Action}, utils::actix::extract_session_key, @@ -246,7 +246,7 @@ where fn call(&self, req: ServiceRequest) -> Self::Future { let username = req.match_info().get("username").unwrap_or(""); - let is_root = username == CONFIG.options.username; + let is_root = username == PARSEABLE.options.username; let fut = self.service.call(req); Box::pin(async move { @@ -300,7 +300,7 @@ where fn call(&self, req: ServiceRequest) -> Self::Future { let path = req.path(); - let mode = &CONFIG.options.mode; + let mode = &PARSEABLE.options.mode; // change error messages based on mode match mode { Mode::Query => { diff --git a/src/handlers/http/mod.rs b/src/handlers/http/mod.rs index 30b0e14c5..f1f702d4b 100644 --- a/src/handlers/http/mod.rs +++ b/src/handlers/http/mod.rs @@ -23,7 +23,7 @@ use http::StatusCode; use itertools::Itertools; use serde_json::Value; -use crate::{option::CONFIG, storage::STREAM_ROOT_DIRECTORY, HTTP_CLIENT}; +use crate::{parseable::PARSEABLE, storage::STREAM_ROOT_DIRECTORY, HTTP_CLIENT}; use self::{cluster::get_ingestor_info, query::Query}; @@ -57,7 +57,7 @@ pub fn metrics_path() -> String { } pub(crate) fn cross_origin_config() -> Cors { - if !CONFIG.options.cors || cfg!(debug_assertions) { + if !PARSEABLE.options.cors || cfg!(debug_assertions) { Cors::permissive().block_on_origin_mismatch(false) } else { Cors::default().block_on_origin_mismatch(false) @@ -80,7 +80,7 @@ pub fn base_path_without_preceding_slash() -> String { pub async fn fetch_schema(stream_name: &str) -> anyhow::Result { let path_prefix = relative_path::RelativePathBuf::from(format!("{}/{}", stream_name, STREAM_ROOT_DIRECTORY)); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let res: Vec = store .get_objects( Some(&path_prefix), diff --git a/src/handlers/http/modal/ingest/ingestor_ingest.rs b/src/handlers/http/modal/ingest/ingestor_ingest.rs index e91a27614..847eef40d 100644 --- a/src/handlers/http/modal/ingest/ingestor_ingest.rs +++ b/src/handlers/http/modal/ingest/ingestor_ingest.rs @@ -19,7 +19,7 @@ use actix_web::{HttpRequest, HttpResponse}; use bytes::Bytes; -use crate::{handlers::http::{ingest::PostError, modal::utils::ingest_utils::flatten_and_push_logs}, metadata::STREAM_INFO}; +use crate::{handlers::http::{ingest::PostError, modal::utils::ingest_utils::flatten_and_push_logs}, metadata::PARSEABLE.streams}; // Handler for POST /api/v1/logstream/{logstream} @@ -27,14 +27,14 @@ use crate::{handlers::http::{ingest::PostError, modal::utils::ingest_utils::flat // fails if the logstream does not exist pub async fn post_event(req: HttpRequest, body: Bytes) -> Result { let stream_name: String = req.match_info().get("logstream").unwrap().parse().unwrap(); - let internal_stream_names = STREAM_INFO.list_internal_streams(); + let internal_stream_names = PARSEABLE.streams.list_internal_streams(); if internal_stream_names.contains(&stream_name) { return Err(PostError::Invalid(anyhow::anyhow!( "Stream {} is an internal stream and cannot be ingested into", stream_name ))); } - if !STREAM_INFO.stream_exists(&stream_name) { + if !PARSEABLE.streams.stream_exists(&stream_name) { return Err(PostError::StreamNotFound(stream_name)); } diff --git a/src/handlers/http/modal/ingest/ingestor_logstream.rs b/src/handlers/http/modal/ingest/ingestor_logstream.rs index d8f80ef4f..4872a6476 100644 --- a/src/handlers/http/modal/ingest/ingestor_logstream.rs +++ b/src/handlers/http/modal/ingest/ingestor_logstream.rs @@ -26,15 +26,8 @@ use tracing::warn; use crate::{ catalog::remove_manifest_from_snapshot, - handlers::http::{ - logstream::error::StreamError, - modal::utils::logstream_utils::{ - create_stream_and_schema_from_storage, create_update_stream, - }, - }, - metadata, - option::CONFIG, - staging::STAGING, + handlers::http::logstream::error::StreamError, + parseable::{StreamNotFound, PARSEABLE}, stats, }; @@ -43,16 +36,17 @@ pub async fn retention_cleanup( Json(date_list): Json>, ) -> Result { let stream_name = stream_name.into_inner(); - let storage = CONFIG.storage().get_object_store(); + let storage = PARSEABLE.storage.get_object_store(); // if the stream not found in memory map, //check if it exists in the storage //create stream and schema from storage - if !metadata::STREAM_INFO.stream_exists(&stream_name) - && !create_stream_and_schema_from_storage(&stream_name) + if !PARSEABLE.streams.contains(&stream_name) + && !PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) .await .unwrap_or(false) { - return Err(StreamError::StreamNotFound(stream_name.clone())); + return Err(StreamNotFound(stream_name.clone()).into()); } let res = remove_manifest_from_snapshot(storage.clone(), &stream_name, date_list).await; @@ -66,16 +60,17 @@ pub async fn delete(stream_name: Path) -> Result Result { let stream_name = stream_name.into_inner(); - create_update_stream(req.headers(), &body, &stream_name).await?; + PARSEABLE + .create_update_stream(req.headers(), &body, &stream_name) + .await?; Ok(("Log stream created", StatusCode::OK)) } diff --git a/src/handlers/http/modal/ingest_server.rs b/src/handlers/http/modal/ingest_server.rs index f60f54c54..46f9ebd83 100644 --- a/src/handlers/http/modal/ingest_server.rs +++ b/src/handlers/http/modal/ingest_server.rs @@ -18,138 +18,39 @@ use std::thread; -use super::ingest::ingestor_logstream; -use super::ingest::ingestor_rbac; -use super::ingest::ingestor_role; -use super::server::Server; -use super::IngestorMetadata; -use super::OpenIdClient; -use super::ParseableServer; -use super::DEFAULT_VERSION; -use crate::analytics; -use crate::handlers::airplane; -use crate::handlers::http::ingest; -use crate::handlers::http::logstream; -use crate::handlers::http::middleware::DisAllowRootUser; -use crate::handlers::http::middleware::RouteExt; -use crate::handlers::http::role; -use crate::migration; -use crate::migration::metadata_migration::migrate_ingester_metadata; -use crate::rbac::role::Action; -use crate::storage::object_storage::ingestor_metadata_path; -use crate::storage::object_storage::parseable_json_path; -use crate::storage::ObjectStorageError; -use crate::storage::PARSEABLE_ROOT_DIRECTORY; -use crate::sync; - -use crate::utils::get_ingestor_id; -use crate::utils::get_url; -use crate::{handlers::http::base_path, option::CONFIG}; - use actix_web::web; use actix_web::Scope; use actix_web_prometheus::PrometheusMetrics; use async_trait::async_trait; use base64::Engine; use bytes::Bytes; -use once_cell::sync::Lazy; use relative_path::RelativePathBuf; use serde_json::Value; use tokio::sync::oneshot; -use tracing::info; - -/// Metadata associated with this ingestor server -pub static INGESTOR_META: Lazy = Lazy::new(|| { - // all the files should be in the staging directory root - let entries = - std::fs::read_dir(&CONFIG.options.local_staging_path).expect("Couldn't read from file"); - let url = get_url(); - let port = url.port().unwrap_or(80).to_string(); - let url = url.to_string(); - - for entry in entries { - // cause the staging directory will have only one file with ingestor in the name - // so the JSON Parse should not error unless the file is corrupted - let path = entry.expect("Should be a directory entry").path(); - let flag = path - .file_name() - .unwrap_or_default() - .to_str() - .unwrap_or_default() - .contains("ingestor"); - - if flag { - // get the ingestor metadata from staging - let text = std::fs::read(path).expect("File should be present"); - let mut meta: Value = serde_json::from_slice(&text).expect("Valid JSON"); - - // migrate the staging meta - let obj = meta - .as_object_mut() - .expect("Could Not parse Ingestor Metadata Json"); - - if obj.get("flight_port").is_none() { - obj.insert( - "flight_port".to_owned(), - Value::String(CONFIG.options.flight_port.to_string()), - ); - } - - let mut meta: IngestorMetadata = - serde_json::from_value(meta).expect("Couldn't write to disk"); - // compare url endpoint and port - if meta.domain_name != url { - info!( - "Domain Name was Updated. Old: {} New: {}", - meta.domain_name, url - ); - meta.domain_name = url; - } - - if meta.port != port { - info!("Port was Updated. Old: {} New: {}", meta.port, port); - meta.port = port; - } - - let token = base64::prelude::BASE64_STANDARD.encode(format!( - "{}:{}", - CONFIG.options.username, CONFIG.options.password - )); - - let token = format!("Basic {}", token); - - if meta.token != token { - // TODO: Update the message to be more informative with username and password - info!( - "Credentials were Updated. Old: {} New: {}", - meta.token, token - ); - meta.token = token; - } - - meta.put_on_disk(CONFIG.staging_dir()) - .expect("Couldn't write to disk"); - return meta; - } - } - - let store = CONFIG.storage().get_object_store(); - let out = IngestorMetadata::new( - port, - url, - DEFAULT_VERSION.to_string(), - store.get_bucket_name(), - &CONFIG.options.username, - &CONFIG.options.password, - get_ingestor_id(), - CONFIG.options.flight_port.to_string(), - ); - - out.put_on_disk(CONFIG.staging_dir()) - .expect("Should Be valid Json"); - out -}); +use crate::{ + analytics, + handlers::{ + airplane, + http::{ + base_path, ingest, logstream, + middleware::{DisAllowRootUser, RouteExt}, + role, + }, + }, + migration, + parseable::PARSEABLE, + rbac::role::Action, + storage::{object_storage::parseable_json_path, ObjectStorageError, PARSEABLE_ROOT_DIRECTORY}, + sync, Server, +}; + +use super::{ + ingest::{ingestor_logstream, ingestor_rbac, ingestor_role}, + OpenIdClient, ParseableServer, +}; + +pub const INGESTOR_EXPECT: &str = "Ingestor Metadata should be set in ingestor mode"; pub struct IngestServer; @@ -176,7 +77,7 @@ impl ParseableServer for IngestServer { async fn load_metadata(&self) -> anyhow::Result> { // parseable can't use local storage for persistence when running a distributed setup - if CONFIG.get_storage_mode_string() == "Local drive" { + if PARSEABLE.storage.name() == "drive" { return Err(anyhow::Error::msg( "This instance of the Parseable server has been configured to run in a distributed setup, it doesn't support local storage.", )); @@ -196,9 +97,9 @@ impl ParseableServer for IngestServer { prometheus: &PrometheusMetrics, shutdown_rx: oneshot::Receiver<()>, ) -> anyhow::Result<()> { - CONFIG.storage().register_store_metrics(prometheus); + PARSEABLE.storage.register_store_metrics(prometheus); - migration::run_migration(&CONFIG).await?; + migration::run_migration(&PARSEABLE).await?; // Run sync on a background thread let (cancel_tx, cancel_rx) = oneshot::channel(); @@ -206,8 +107,8 @@ impl ParseableServer for IngestServer { tokio::spawn(airplane::server()); - // set the ingestor metadata - set_ingestor_metadata().await?; + // write the ingestor metadata to storage + PARSEABLE.store_ingestor_metadata().await?; // Ingestors shouldn't have to deal with OpenId auth flow let result = self.start(shutdown_rx, prometheus.clone(), None).await; @@ -348,47 +249,14 @@ impl IngestServer { } } -// create the ingestor metadata and put the .ingestor.json file in the object store -pub async fn set_ingestor_metadata() -> anyhow::Result<()> { - let storage_ingestor_metadata = migrate_ingester_metadata().await?; - let store = CONFIG.storage().get_object_store(); - - // find the meta file in staging if not generate new metadata - let resource = INGESTOR_META.clone(); - // use the id that was generated/found in the staging and - // generate the path for the object store - let path = ingestor_metadata_path(None); - - // we are considering that we can always get from object store - if let Some(mut store_data) = storage_ingestor_metadata { - if store_data.domain_name != INGESTOR_META.domain_name { - store_data - .domain_name - .clone_from(&INGESTOR_META.domain_name); - store_data.port.clone_from(&INGESTOR_META.port); - - let resource = Bytes::from(serde_json::to_vec(&store_data)?); - - // if pushing to object store fails propagate the error - store.put_object(&path, resource).await?; - } - } else { - let resource = Bytes::from(serde_json::to_vec(&resource)?); - - store.put_object(&path, resource).await?; - } - - Ok(()) -} - // check for querier state. Is it there, or was it there in the past // this should happen before the set the ingestor metadata async fn check_querier_state() -> anyhow::Result, ObjectStorageError> { // how do we check for querier state? // based on the work flow of the system, the querier will always need to start first // i.e the querier will create the `.parseable.json` file - let parseable_json = CONFIG - .storage() + let parseable_json = PARSEABLE + .storage .get_object_store() .get_object(&parseable_json_path()) .await @@ -404,7 +272,7 @@ async fn check_querier_state() -> anyhow::Result, ObjectStorageErr async fn validate_credentials() -> anyhow::Result<()> { // check if your creds match with others - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let base_path = RelativePathBuf::from(PARSEABLE_ROOT_DIRECTORY); let ingestor_metadata = store .get_objects( @@ -423,7 +291,7 @@ async fn validate_credentials() -> anyhow::Result<()> { let token = base64::prelude::BASE64_STANDARD.encode(format!( "{}:{}", - CONFIG.options.username, CONFIG.options.password + PARSEABLE.options.username, PARSEABLE.options.password )); let token = format!("Basic {}", token); diff --git a/src/handlers/http/modal/mod.rs b/src/handlers/http/modal/mod.rs index 6cc1ec0c9..6b5711079 100644 --- a/src/handlers/http/modal/mod.rs +++ b/src/handlers/http/modal/mod.rs @@ -16,42 +16,40 @@ * */ -pub mod ingest; -pub mod ingest_server; -pub mod query; -pub mod query_server; -pub mod server; -pub mod ssl_acceptor; -pub mod utils; - -use std::path::Path; -use std::sync::Arc; +use std::{path::Path, sync::Arc}; -use actix_web::middleware::from_fn; -use actix_web::web::ServiceConfig; -use actix_web::App; -use actix_web::HttpServer; +use actix_web::{middleware::from_fn, web::ServiceConfig, App, HttpServer}; use actix_web_prometheus::PrometheusMetrics; use async_trait::async_trait; use base64::Engine; use bytes::Bytes; use openid::Discovered; -use serde::Deserialize; -use serde::Serialize; +use relative_path::RelativePathBuf; +use serde::{Deserialize, Serialize}; +use serde_json::Value; use ssl_acceptor::get_ssl_acceptor; use tokio::sync::oneshot; use tracing::{error, info, warn}; -use super::audit; -use super::cross_origin_config; -use super::API_BASE_PATH; -use super::API_VERSION; -use crate::handlers::http::health_check; -use crate::oidc; -use crate::option::CONFIG; -use crate::staging::STAGING; +use crate::{ + cli::Options, + oidc::Claims, + parseable::PARSEABLE, + storage::PARSEABLE_ROOT_DIRECTORY, + utils::{get_ingestor_id, get_url}, +}; -pub type OpenIdClient = Arc>; +use super::{audit, cross_origin_config, health_check, API_BASE_PATH, API_VERSION}; + +pub mod ingest; +pub mod ingest_server; +pub mod query; +pub mod query_server; +pub mod server; +pub mod ssl_acceptor; +pub mod utils; + +pub type OpenIdClient = Arc>; // to be decided on what the Default version should be pub const DEFAULT_VERSION: &str = "v3"; @@ -98,9 +96,9 @@ pub trait ParseableServer { // get the ssl stuff let ssl = get_ssl_acceptor( - &CONFIG.options.tls_cert_path, - &CONFIG.options.tls_key_path, - &CONFIG.options.trusted_ca_certs_path, + &PARSEABLE.options.tls_cert_path, + &PARSEABLE.options.tls_key_path, + &PARSEABLE.options.trusted_ca_certs_path, )?; // fn that creates the app @@ -123,10 +121,10 @@ pub trait ParseableServer { // Start the server with or without TLS let srv = if let Some(config) = ssl { http_server - .bind_rustls_0_22(&CONFIG.options.address, config)? + .bind_rustls_0_22(&PARSEABLE.options.address, config)? .run() } else { - http_server.bind(&CONFIG.options.address)?.run() + http_server.bind(&PARSEABLE.options.address)?.run() }; // Graceful shutdown handling @@ -141,14 +139,14 @@ pub trait ParseableServer { // Perform S3 sync and wait for completion info!("Starting data sync to S3..."); - if let Err(e) = STAGING.prepare_parquet(true) { + if let Err(e) = PARSEABLE.streams.prepare_parquet(true) { warn!("Failed to convert arrow files to parquet. {:?}", e); } else { info!("Successfully converted arrow files to parquet."); } - if let Err(e) = CONFIG - .storage() + if let Err(e) = PARSEABLE + .storage .get_object_store() .upload_files_from_staging() .await @@ -180,7 +178,7 @@ pub trait ParseableServer { } } -#[derive(Serialize, Debug, Deserialize, Default, Clone, Eq, PartialEq)] +#[derive(Debug, Serialize, Deserialize, Default, Clone, Eq, PartialEq)] pub struct IngestorMetadata { pub version: String, pub port: String, @@ -192,36 +190,169 @@ pub struct IngestorMetadata { } impl IngestorMetadata { - #[allow(clippy::too_many_arguments)] pub fn new( port: String, domain_name: String, - version: String, bucket_name: String, username: &str, password: &str, ingestor_id: String, flight_port: String, ) -> Self { - let token = base64::prelude::BASE64_STANDARD.encode(format!("{}:{}", username, password)); - - let token = format!("Basic {}", token); + let token = base64::prelude::BASE64_STANDARD.encode(format!("{username}:{password}")); Self { port, domain_name, - version, + version: DEFAULT_VERSION.to_string(), bucket_name, - token, + token: format!("Basic {token}"), ingestor_id, flight_port, } } + /// Capture metadata information by either loading it from staging or starting fresh + pub fn load() -> Arc { + // all the files should be in the staging directory root + let entries = std::fs::read_dir(&PARSEABLE.options.local_staging_path) + .expect("Couldn't read from file"); + let url = get_url(); + let port = url.port().unwrap_or(80).to_string(); + let url = url.to_string(); + let Options { + username, password, .. + } = PARSEABLE.options.as_ref(); + let staging_path = PARSEABLE.staging_dir(); + let flight_port = PARSEABLE.options.flight_port.to_string(); + + for entry in entries { + // cause the staging directory will have only one file with ingestor in the name + // so the JSON Parse should not error unless the file is corrupted + let path = entry.expect("Should be a directory entry").path(); + let flag = path + .file_name() + .unwrap_or_default() + .to_str() + .unwrap_or_default() + .contains("ingestor"); + + if flag { + // get the ingestor metadata from staging + let text = std::fs::read(path).expect("File should be present"); + let mut meta: Value = serde_json::from_slice(&text).expect("Valid JSON"); + + // migrate the staging meta + let obj = meta + .as_object_mut() + .expect("Could Not parse Ingestor Metadata Json"); + + if obj.get("flight_port").is_none() { + obj.insert( + "flight_port".to_owned(), + Value::String(PARSEABLE.options.flight_port.to_string()), + ); + } + + let mut meta: IngestorMetadata = + serde_json::from_value(meta).expect("Couldn't write to disk"); + + // compare url endpoint and port + if meta.domain_name != url { + info!( + "Domain Name was Updated. Old: {} New: {}", + meta.domain_name, url + ); + meta.domain_name = url; + } + + if meta.port != port { + info!("Port was Updated. Old: {} New: {}", meta.port, port); + meta.port = port; + } + + let token = + base64::prelude::BASE64_STANDARD.encode(format!("{}:{}", username, password)); + + let token = format!("Basic {}", token); + + if meta.token != token { + // TODO: Update the message to be more informative with username and password + info!( + "Credentials were Updated. Old: {} New: {}", + meta.token, token + ); + meta.token = token; + } + + meta.put_on_disk(staging_path) + .expect("Couldn't write to disk"); + return Arc::new(meta); + } + } + + let storage = PARSEABLE.storage.get_object_store(); + let meta = Self::new( + port, + url, + storage.get_bucket_name(), + username, + password, + get_ingestor_id(), + flight_port, + ); + + meta.put_on_disk(staging_path) + .expect("Should Be valid Json"); + Arc::new(meta) + } + pub fn get_ingestor_id(&self) -> String { self.ingestor_id.clone() } + #[inline(always)] + pub fn file_path(&self) -> RelativePathBuf { + RelativePathBuf::from_iter([ + PARSEABLE_ROOT_DIRECTORY, + &format!("ingestor.{}.json", self.get_ingestor_id()), + ]) + } + + pub async fn migrate(&self) -> anyhow::Result> { + let imp = self.file_path(); + let bytes = match PARSEABLE.storage.get_object_store().get_object(&imp).await { + Ok(bytes) => bytes, + Err(_) => { + return Ok(None); + } + }; + let mut json = serde_json::from_slice::(&bytes)?; + let meta = json + .as_object_mut() + .ok_or_else(|| anyhow::anyhow!("Unable to parse Ingester Metadata"))?; + let fp = meta.get("flight_port"); + + if fp.is_none() { + meta.insert( + "flight_port".to_owned(), + Value::String(PARSEABLE.options.flight_port.to_string()), + ); + } + let bytes = Bytes::from(serde_json::to_vec(&json)?); + + let resource: IngestorMetadata = serde_json::from_value(json)?; + resource.put_on_disk(PARSEABLE.staging_dir())?; + + PARSEABLE + .storage + .get_object_store() + .put_object(&imp, bytes) + .await?; + + Ok(Some(resource)) + } + /// Puts the ingestor info into the staging. /// /// This function takes the ingestor info as a parameter and stores it in staging. @@ -244,18 +375,17 @@ mod test { use bytes::Bytes; use rstest::rstest; - use super::{IngestorMetadata, DEFAULT_VERSION}; + use super::IngestorMetadata; #[rstest] fn test_deserialize_resource() { let lhs: IngestorMetadata = IngestorMetadata::new( "8000".to_string(), "https://localhost:8000".to_string(), - DEFAULT_VERSION.to_string(), "somebucket".to_string(), "admin", "admin", - "ingestor_id".to_string(), + "ingestor_id".to_owned(), "8002".to_string(), ); @@ -269,11 +399,10 @@ mod test { let im = IngestorMetadata::new( "8000".to_string(), "https://localhost:8000".to_string(), - DEFAULT_VERSION.to_string(), "somebucket".to_string(), "admin", "admin", - "ingestor_id".to_string(), + "ingestor_id".to_owned(), "8002".to_string(), ); diff --git a/src/handlers/http/modal/query/querier_logstream.rs b/src/handlers/http/modal/query/querier_logstream.rs index 1b41660a6..4c796d0a1 100644 --- a/src/handlers/http/modal/query/querier_logstream.rs +++ b/src/handlers/http/modal/query/querier_logstream.rs @@ -40,16 +40,12 @@ use crate::{ utils::{merge_quried_stats, IngestionStats, QueriedStats, StorageStats}, }, logstream::{error::StreamError, get_stats_date}, - modal::utils::logstream_utils::{ - create_stream_and_schema_from_storage, create_update_stream, - }, }, hottier::HotTierManager, - metadata::{self, STREAM_INFO}, - option::CONFIG, - staging::{Stream, STAGING}, + parseable::{StreamNotFound, PARSEABLE}, stats::{self, Stats}, storage::StreamType, + LOCK_EXPECT, }; pub async fn delete(stream_name: Path) -> Result { @@ -58,18 +54,19 @@ pub async fn delete(stream_name: Path) -> Result) -> Result) -> Result Result { let stream_name = stream_name.into_inner(); let _ = CREATE_STREAM_LOCK.lock().await; - let headers = create_update_stream(req.headers(), &body, &stream_name).await?; + let headers = PARSEABLE + .create_update_stream(req.headers(), &body, &stream_name) + .await?; sync_streams_with_ingestors(headers, body, &stream_name).await?; @@ -131,12 +130,13 @@ pub async fn get_stats( // if the stream not found in memory map, //check if it exists in the storage //create stream and schema from storage - if !metadata::STREAM_INFO.stream_exists(&stream_name) - && !create_stream_and_schema_from_storage(&stream_name) + if !PARSEABLE.streams.contains(&stream_name) + && !PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) .await .unwrap_or(false) { - return Err(StreamError::StreamNotFound(stream_name.clone())); + return Err(StreamNotFound(stream_name.clone()).into()); } let query_string = req.query_string(); @@ -165,21 +165,24 @@ pub async fn get_stats( } let stats = stats::get_current_stats(&stream_name, "json") - .ok_or(StreamError::StreamNotFound(stream_name.clone()))?; + .ok_or_else(|| StreamNotFound(stream_name.clone()))?; - let ingestor_stats = if STREAM_INFO - .stream_type(&stream_name) - .is_ok_and(|t| t == StreamType::UserDefined) + let ingestor_stats = if PARSEABLE + .get_stream(&stream_name) + .is_ok_and(|stream| stream.get_stream_type() == StreamType::UserDefined) { Some(fetch_stats_from_ingestors(&stream_name).await?) } else { None }; - let hash_map = STREAM_INFO.read().expect("Readable"); - let stream_meta = &hash_map + let hash_map = PARSEABLE.streams.read().expect(LOCK_EXPECT); + let stream_meta = hash_map .get(&stream_name) - .ok_or(StreamError::StreamNotFound(stream_name.clone()))?; + .ok_or_else(|| StreamNotFound(stream_name.clone()))? + .metadata + .read() + .expect(LOCK_EXPECT); let time = Utc::now(); diff --git a/src/handlers/http/modal/query_server.rs b/src/handlers/http/modal/query_server.rs index 938e05dc5..25dd11aa2 100644 --- a/src/handlers/http/modal/query_server.rs +++ b/src/handlers/http/modal/query_server.rs @@ -23,10 +23,9 @@ use crate::correlation::CORRELATIONS; use crate::handlers::airplane; use crate::handlers::http::base_path; use crate::handlers::http::cluster::{self, init_cluster_metrics_schedular}; -use crate::handlers::http::logstream::create_internal_stream_if_not_exists; use crate::handlers::http::middleware::{DisAllowRootUser, RouteExt}; -use crate::handlers::http::{self, role}; use crate::handlers::http::{logstream, MAX_EVENT_PAYLOAD_SIZE}; +use crate::handlers::http::{rbac, role}; use crate::hottier::HotTierManager; use crate::rbac::role::Action; use crate::users::dashboards::DASHBOARDS; @@ -40,11 +39,11 @@ use bytes::Bytes; use tokio::sync::oneshot; use tracing::{error, info}; -use crate::{option::CONFIG, ParseableServer}; +use crate::parseable::PARSEABLE; +use crate::Server; use super::query::{querier_ingest, querier_logstream, querier_rbac, querier_role}; -use super::server::Server; -use super::OpenIdClient; +use super::{OpenIdClient, ParseableServer}; pub struct QueryServer; @@ -77,15 +76,15 @@ impl ParseableServer for QueryServer { async fn load_metadata(&self) -> anyhow::Result> { // parseable can't use local storage for persistence when running a distributed setup - if CONFIG.get_storage_mode_string() == "Local drive" { + if PARSEABLE.storage.name() == "drive" { return Err(anyhow::anyhow!( "This instance of the Parseable server has been configured to run in a distributed setup, it doesn't support local storage.", )); } - migration::run_file_migration(&CONFIG).await?; - let parseable_json = CONFIG.validate_storage().await?; - migration::run_metadata_migration(&CONFIG, &parseable_json).await?; + migration::run_file_migration(&PARSEABLE).await?; + let parseable_json = PARSEABLE.validate_storage().await?; + migration::run_metadata_migration(&PARSEABLE, &parseable_json).await?; Ok(parseable_json) } @@ -96,12 +95,12 @@ impl ParseableServer for QueryServer { prometheus: &PrometheusMetrics, shutdown_rx: oneshot::Receiver<()>, ) -> anyhow::Result<()> { - CONFIG.storage().register_store_metrics(prometheus); + PARSEABLE.storage.register_store_metrics(prometheus); - migration::run_migration(&CONFIG).await?; + migration::run_migration(&PARSEABLE).await?; //create internal stream at server start - create_internal_stream_if_not_exists().await?; + PARSEABLE.create_internal_stream_if_not_exists().await?; if let Err(e) = CORRELATIONS.load().await { error!("{e}"); @@ -123,11 +122,11 @@ impl ParseableServer for QueryServer { // all internal data structures populated now. // start the analytics scheduler if enabled - if CONFIG.options.send_analytics { + if PARSEABLE.options.send_analytics { analytics::init_analytics_scheduler()?; } - if matches!(init_cluster_metrics_schedular(), Ok(())) { + if init_cluster_metrics_schedular().is_ok() { info!("Cluster metrics scheduler started successfully"); } if let Some(hot_tier_manager) = HotTierManager::global() { @@ -176,11 +175,7 @@ impl QueryServer { .service( web::resource("") // GET /user => List all users - .route( - web::get() - .to(http::rbac::list_users) - .authorize(Action::ListUser), - ), + .route(web::get().to(rbac::list_users).authorize(Action::ListUser)), ) .service( web::resource("/{username}") @@ -209,7 +204,7 @@ impl QueryServer { ) .route( web::get() - .to(http::rbac::get_role) + .to(rbac::get_role) .authorize_for_user(Action::GetUserRoles), ), ) diff --git a/src/handlers/http/modal/server.rs b/src/handlers/http/modal/server.rs index 4493a954a..aef35fcc7 100644 --- a/src/handlers/http/modal/server.rs +++ b/src/handlers/http/modal/server.rs @@ -54,7 +54,7 @@ use crate::{ middleware::{DisAllowRootUser, RouteExt}, oidc, role, MAX_EVENT_PAYLOAD_SIZE, }, - option::CONFIG, + parseable::PARSEABLE, rbac::role::Action, }; @@ -94,9 +94,9 @@ impl ParseableServer for Server { } async fn load_metadata(&self) -> anyhow::Result> { - migration::run_file_migration(&CONFIG).await?; - let parseable_json = CONFIG.validate_storage().await?; - migration::run_metadata_migration(&CONFIG, &parseable_json).await?; + migration::run_file_migration(&PARSEABLE).await?; + let parseable_json = PARSEABLE.validate_storage().await?; + migration::run_metadata_migration(&PARSEABLE, &parseable_json).await?; Ok(parseable_json) } @@ -107,9 +107,9 @@ impl ParseableServer for Server { prometheus: &PrometheusMetrics, shutdown_rx: oneshot::Receiver<()>, ) -> anyhow::Result<()> { - CONFIG.storage().register_store_metrics(prometheus); + PARSEABLE.storage.register_store_metrics(prometheus); - migration::run_migration(&CONFIG).await?; + migration::run_migration(&PARSEABLE).await?; if let Err(e) = CORRELATIONS.load().await { error!("{e}"); @@ -136,7 +136,7 @@ impl ParseableServer for Server { let (cancel_tx, cancel_rx) = oneshot::channel(); thread::spawn(|| sync::handler(cancel_rx)); - if CONFIG.options.send_analytics { + if PARSEABLE.options.send_analytics { analytics::init_analytics_scheduler()?; } @@ -144,7 +144,7 @@ impl ParseableServer for Server { tokio::spawn(handlers::airplane::server()); let result = self - .start(shutdown_rx, prometheus.clone(), CONFIG.options.openid()) + .start(shutdown_rx, prometheus.clone(), PARSEABLE.options.openid()) .await; // Cancel sync jobs cancel_tx.send(()).expect("Cancellation should not fail"); diff --git a/src/handlers/http/modal/utils/ingest_utils.rs b/src/handlers/http/modal/utils/ingest_utils.rs index 3a2b9c797..402f4d3df 100644 --- a/src/handlers/http/modal/utils/ingest_utils.rs +++ b/src/handlers/http/modal/utils/ingest_utils.rs @@ -31,9 +31,11 @@ use crate::{ ingest::PostError, kinesis::{flatten_kinesis_logs, Message}, }, - metadata::{SchemaVersion, STREAM_INFO}, + metadata::SchemaVersion, + parseable::{StreamNotFound, PARSEABLE}, storage::StreamType, utils::json::{convert_array_to_object, flatten::convert_to_array}, + LOCK_EXPECT, }; pub async fn flatten_and_push_logs( @@ -64,11 +66,14 @@ pub async fn push_logs( json: Value, log_source: &LogSource, ) -> Result<(), PostError> { - let time_partition = STREAM_INFO.get_time_partition(stream_name)?; - let time_partition_limit = STREAM_INFO.get_time_partition_limit(stream_name)?; - let static_schema_flag = STREAM_INFO.get_static_schema_flag(stream_name)?; - let custom_partition = STREAM_INFO.get_custom_partition(stream_name)?; - let schema_version = STREAM_INFO.get_schema_version(stream_name)?; + let stream = PARSEABLE.get_stream(stream_name)?; + let time_partition = stream.get_time_partition(); + let time_partition_limit = PARSEABLE + .get_stream(stream_name)? + .get_time_partition_limit(); + let static_schema_flag = stream.get_static_schema_flag(); + let custom_partition = stream.get_custom_partition(); + let schema_version = stream.get_schema_version(); let data = if time_partition.is_some() || custom_partition.is_some() { convert_array_to_object( @@ -103,11 +108,15 @@ pub async fn push_logs( } None => HashMap::new(), }; - let schema = STREAM_INFO + let schema = PARSEABLE + .streams .read() .unwrap() .get(stream_name) - .ok_or(PostError::StreamNotFound(stream_name.to_owned()))? + .ok_or_else(|| StreamNotFound(stream_name.to_owned()))? + .metadata + .read() + .expect(LOCK_EXPECT) .schema .clone(); let (rb, is_first_event) = into_event_batch( diff --git a/src/handlers/http/modal/utils/logstream_utils.rs b/src/handlers/http/modal/utils/logstream_utils.rs index f6cf6d6b9..15c25da2b 100644 --- a/src/handlers/http/modal/utils/logstream_utils.rs +++ b/src/handlers/http/modal/utils/logstream_utils.rs @@ -16,164 +16,22 @@ * */ -use std::{collections::HashMap, num::NonZeroU32, sync::Arc}; - use actix_web::http::header::HeaderMap; -use arrow_schema::{Field, Schema}; -use bytes::Bytes; -use http::StatusCode; use crate::{ event::format::LogSource, handlers::{ - http::logstream::error::{CreateStreamError, StreamError}, CUSTOM_PARTITION_KEY, LOG_SOURCE_KEY, STATIC_SCHEMA_FLAG, STREAM_TYPE_KEY, TIME_PARTITION_KEY, TIME_PARTITION_LIMIT_KEY, UPDATE_STREAM_KEY, }, - metadata::{self, SchemaVersion, STREAM_INFO}, - option::{Mode, CONFIG}, - static_schema::{convert_static_schema_to_arrow_schema, StaticSchema}, - storage::{ObjectStoreFormat, StreamType}, - validator, + storage::StreamType, }; -use tracing::error; - -pub async fn create_update_stream( - headers: &HeaderMap, - body: &Bytes, - stream_name: &str, -) -> Result { - let PutStreamHeaders { - time_partition, - time_partition_limit, - custom_partition, - static_schema_flag, - update_stream_flag, - stream_type, - log_source, - } = headers.into(); - - if metadata::STREAM_INFO.stream_exists(stream_name) && !update_stream_flag { - return Err(StreamError::Custom { - msg: format!( - "Logstream {stream_name} already exists, please create a new log stream with unique name" - ), - status: StatusCode::BAD_REQUEST, - }); - } - - if !metadata::STREAM_INFO.stream_exists(stream_name) - && CONFIG.options.mode == Mode::Query - && create_stream_and_schema_from_storage(stream_name).await? - { - return Err(StreamError::Custom { - msg: format!( - "Logstream {stream_name} already exists, please create a new log stream with unique name" - ), - status: StatusCode::BAD_REQUEST, - }); - } - - if update_stream_flag { - return update_stream( - headers, - stream_name, - &time_partition, - static_schema_flag, - &time_partition_limit, - &custom_partition, - ) - .await; - } - - let time_partition_in_days = if !time_partition_limit.is_empty() { - Some(validate_time_partition_limit(&time_partition_limit)?) - } else { - None - }; - - if !custom_partition.is_empty() { - validate_custom_partition(&custom_partition)?; - } - - if !time_partition.is_empty() && !custom_partition.is_empty() { - validate_time_with_custom_partition(&time_partition, &custom_partition)?; - } - - let schema = validate_static_schema( - body, - stream_name, - &time_partition, - &custom_partition, - static_schema_flag, - )?; - - create_stream( - stream_name.to_string(), - &time_partition, - time_partition_in_days, - &custom_partition, - static_schema_flag, - schema, - stream_type, - log_source, - ) - .await?; - - Ok(headers.clone()) -} - -async fn update_stream( - headers: &HeaderMap, - stream_name: &str, - time_partition: &str, - static_schema_flag: bool, - time_partition_limit: &str, - custom_partition: &str, -) -> Result { - if !STREAM_INFO.stream_exists(stream_name) { - return Err(StreamError::StreamNotFound(stream_name.to_string())); - } - if !time_partition.is_empty() { - return Err(StreamError::Custom { - msg: "Altering the time partition of an existing stream is restricted.".to_string(), - status: StatusCode::BAD_REQUEST, - }); - } - if static_schema_flag { - return Err(StreamError::Custom { - msg: "Altering the schema of an existing stream is restricted.".to_string(), - status: StatusCode::BAD_REQUEST, - }); - } - if !time_partition_limit.is_empty() { - let time_partition_days = validate_time_partition_limit(time_partition_limit)?; - update_time_partition_limit_in_stream(stream_name.to_string(), time_partition_days).await?; - return Ok(headers.clone()); - } - validate_and_update_custom_partition(stream_name, custom_partition).await?; - - Ok(headers.clone()) -} - -async fn validate_and_update_custom_partition( - stream_name: &str, - custom_partition: &str, -) -> Result<(), StreamError> { - if !custom_partition.is_empty() { - validate_custom_partition(custom_partition)?; - update_custom_partition_in_stream(stream_name.to_string(), custom_partition).await?; - } else { - update_custom_partition_in_stream(stream_name.to_string(), "").await?; - } - Ok(()) -} #[derive(Debug, Default)] pub struct PutStreamHeaders { pub time_partition: String, pub time_partition_limit: String, - pub custom_partition: String, + pub custom_partition: Option, pub static_schema_flag: bool, pub update_stream_flag: bool, pub stream_type: StreamType, @@ -193,8 +51,7 @@ impl From<&HeaderMap> for PutStreamHeaders { .to_string(), custom_partition: headers .get(CUSTOM_PARTITION_KEY) - .map_or("", |v| v.to_str().unwrap()) - .to_string(), + .map(|v| v.to_str().unwrap().to_string()), static_schema_flag: headers .get(STATIC_SCHEMA_FLAG) .is_some_and(|v| v.to_str().unwrap() == "true"), @@ -211,349 +68,3 @@ impl From<&HeaderMap> for PutStreamHeaders { } } } - -pub fn validate_time_partition_limit( - time_partition_limit: &str, -) -> Result { - if !time_partition_limit.ends_with('d') { - return Err(CreateStreamError::Custom { - msg: "Missing 'd' suffix for duration value".to_string(), - status: StatusCode::BAD_REQUEST, - }); - } - let days = &time_partition_limit[0..time_partition_limit.len() - 1]; - let Ok(days) = days.parse::() else { - return Err(CreateStreamError::Custom { - msg: "Could not convert duration to an unsigned number".to_string(), - status: StatusCode::BAD_REQUEST, - }); - }; - - Ok(days) -} - -pub fn validate_custom_partition(custom_partition: &str) -> Result<(), CreateStreamError> { - let custom_partition_list = custom_partition.split(',').collect::>(); - if custom_partition_list.len() > 3 { - return Err(CreateStreamError::Custom { - msg: "Maximum 3 custom partition keys are supported".to_string(), - status: StatusCode::BAD_REQUEST, - }); - } - Ok(()) -} - -pub fn validate_time_with_custom_partition( - time_partition: &str, - custom_partition: &str, -) -> Result<(), CreateStreamError> { - let custom_partition_list = custom_partition.split(',').collect::>(); - if custom_partition_list.contains(&time_partition) { - return Err(CreateStreamError::Custom { - msg: format!( - "time partition {} cannot be set as custom partition", - time_partition - ), - status: StatusCode::BAD_REQUEST, - }); - } - Ok(()) -} - -pub fn validate_static_schema( - body: &Bytes, - stream_name: &str, - time_partition: &str, - custom_partition: &str, - static_schema_flag: bool, -) -> Result, CreateStreamError> { - if static_schema_flag { - if body.is_empty() { - return Err(CreateStreamError::Custom { - msg: format!( - "Please provide schema in the request body for static schema logstream {stream_name}" - ), - status: StatusCode::BAD_REQUEST, - }); - } - - let static_schema: StaticSchema = serde_json::from_slice(body)?; - let parsed_schema = - convert_static_schema_to_arrow_schema(static_schema, time_partition, custom_partition) - .map_err(|_| CreateStreamError::Custom { - msg: format!( - "Unable to commit static schema, logstream {stream_name} not created" - ), - status: StatusCode::BAD_REQUEST, - })?; - - return Ok(parsed_schema); - } - - Ok(Arc::new(Schema::empty())) -} - -pub async fn update_time_partition_limit_in_stream( - stream_name: String, - time_partition_limit: NonZeroU32, -) -> Result<(), CreateStreamError> { - let storage = CONFIG.storage().get_object_store(); - if let Err(err) = storage - .update_time_partition_limit_in_stream(&stream_name, time_partition_limit) - .await - { - return Err(CreateStreamError::Storage { stream_name, err }); - } - - if metadata::STREAM_INFO - .update_time_partition_limit(&stream_name, time_partition_limit) - .is_err() - { - return Err(CreateStreamError::Custom { - msg: "failed to update time partition limit in metadata".to_string(), - status: StatusCode::EXPECTATION_FAILED, - }); - } - - Ok(()) -} - -pub async fn update_custom_partition_in_stream( - stream_name: String, - custom_partition: &str, -) -> Result<(), CreateStreamError> { - let static_schema_flag = STREAM_INFO.get_static_schema_flag(&stream_name).unwrap(); - let time_partition = STREAM_INFO.get_time_partition(&stream_name).unwrap(); - if static_schema_flag { - let schema = STREAM_INFO.schema(&stream_name).unwrap(); - - if !custom_partition.is_empty() { - let custom_partition_list = custom_partition.split(',').collect::>(); - let custom_partition_exists: HashMap<_, _> = custom_partition_list - .iter() - .map(|&partition| { - ( - partition.to_string(), - schema - .fields() - .iter() - .any(|field| field.name() == partition), - ) - }) - .collect(); - - for partition in &custom_partition_list { - if !custom_partition_exists[*partition] { - return Err(CreateStreamError::Custom { - msg: format!("custom partition field {} does not exist in the schema for the stream {}", partition, stream_name), - status: StatusCode::BAD_REQUEST, - }); - } - - if let Some(time_partition) = time_partition.clone() { - if time_partition == *partition { - return Err(CreateStreamError::Custom { - msg: format!( - "time partition {} cannot be set as custom partition", - partition - ), - status: StatusCode::BAD_REQUEST, - }); - } - } - } - } - } - let storage = CONFIG.storage().get_object_store(); - if let Err(err) = storage - .update_custom_partition_in_stream(&stream_name, custom_partition) - .await - { - return Err(CreateStreamError::Storage { stream_name, err }); - } - - if metadata::STREAM_INFO - .update_custom_partition(&stream_name, custom_partition.to_string()) - .is_err() - { - return Err(CreateStreamError::Custom { - msg: "failed to update custom partition in metadata".to_string(), - status: StatusCode::EXPECTATION_FAILED, - }); - } - - Ok(()) -} - -#[allow(clippy::too_many_arguments)] -pub async fn create_stream( - stream_name: String, - time_partition: &str, - time_partition_limit: Option, - custom_partition: &str, - static_schema_flag: bool, - schema: Arc, - stream_type: StreamType, - log_source: LogSource, -) -> Result<(), CreateStreamError> { - // fail to proceed if invalid stream name - if stream_type != StreamType::Internal { - validator::stream_name(&stream_name, stream_type)?; - } - // Proceed to create log stream if it doesn't exist - let storage = CONFIG.storage().get_object_store(); - - match storage - .create_stream( - &stream_name, - time_partition, - time_partition_limit, - custom_partition, - static_schema_flag, - schema.clone(), - stream_type, - log_source.clone(), - ) - .await - { - Ok(created_at) => { - let mut static_schema: HashMap> = HashMap::new(); - - for (field_name, field) in schema - .fields() - .iter() - .map(|field| (field.name().to_string(), field.clone())) - { - static_schema.insert(field_name, field); - } - - metadata::STREAM_INFO.add_stream( - stream_name.to_string(), - created_at, - time_partition.to_string(), - time_partition_limit, - custom_partition.to_string(), - static_schema_flag, - static_schema, - stream_type, - SchemaVersion::V1, // New stream - log_source, - ); - } - Err(err) => { - return Err(CreateStreamError::Storage { stream_name, err }); - } - } - Ok(()) -} - -/// list all streams from storage -/// if stream exists in storage, create stream and schema from storage -/// and add it to the memory map -pub async fn create_stream_and_schema_from_storage(stream_name: &str) -> Result { - // Proceed to create log stream if it doesn't exist - let storage = CONFIG.storage().get_object_store(); - let streams = storage.list_streams().await?; - if streams.contains(stream_name) { - let mut stream_metadata = ObjectStoreFormat::default(); - let stream_metadata_bytes = storage.create_stream_from_ingestor(stream_name).await?; - if !stream_metadata_bytes.is_empty() { - stream_metadata = serde_json::from_slice::(&stream_metadata_bytes)?; - } - - let mut schema = Arc::new(Schema::empty()); - let schema_bytes = storage.create_schema_from_ingestor(stream_name).await?; - if !schema_bytes.is_empty() { - schema = serde_json::from_slice::>(&schema_bytes)?; - } - - let mut static_schema: HashMap> = HashMap::new(); - - for (field_name, field) in schema - .fields() - .iter() - .map(|field| (field.name().to_string(), field.clone())) - { - static_schema.insert(field_name, field); - } - - let time_partition = stream_metadata.time_partition.as_deref().unwrap_or(""); - let time_partition_limit = stream_metadata - .time_partition_limit - .and_then(|limit| limit.parse().ok()); - let custom_partition = stream_metadata.custom_partition.as_deref().unwrap_or(""); - let static_schema_flag = stream_metadata.static_schema_flag; - let stream_type = stream_metadata.stream_type; - let schema_version = stream_metadata.schema_version; - let log_source = stream_metadata.log_source; - metadata::STREAM_INFO.add_stream( - stream_name.to_string(), - stream_metadata.created_at, - time_partition.to_string(), - time_partition_limit, - custom_partition.to_string(), - static_schema_flag, - static_schema, - stream_type, - schema_version, - log_source, - ); - } else { - return Ok(false); - } - - Ok(true) -} - -/// Updates the first-event-at in storage and logstream metadata for the specified stream. -/// -/// This function updates the `first-event-at` in both the object store and the stream info metadata. -/// If either update fails, an error is logged, but the function will still return the `first-event-at`. -/// -/// # Arguments -/// -/// * `stream_name` - The name of the stream to update. -/// * `first_event_at` - The value of first-event-at. -/// -/// # Returns -/// -/// * `Option` - Returns `Some(String)` with the provided timestamp if the update is successful, -/// or `None` if an error occurs. -/// -/// # Errors -/// -/// This function logs an error if: -/// * The `first-event-at` cannot be updated in the object store. -/// * The `first-event-at` cannot be updated in the stream info. -/// -/// # Examples -///```ignore -/// ```rust -/// use parseable::handlers::http::modal::utils::logstream_utils::update_first_event_at; -/// let result = update_first_event_at("my_stream", "2023-01-01T00:00:00Z").await; -/// match result { -/// Some(timestamp) => println!("first-event-at: {}", timestamp), -/// None => eprintln!("Failed to update first-event-at"), -/// } -/// ``` -pub async fn update_first_event_at(stream_name: &str, first_event_at: &str) -> Option { - let storage = CONFIG.storage().get_object_store(); - if let Err(err) = storage - .update_first_event_in_stream(stream_name, first_event_at) - .await - { - error!( - "Failed to update first_event_at in storage for stream {:?}: {err:?}", - stream_name - ); - } - - if let Err(err) = metadata::STREAM_INFO.set_first_event_at(stream_name, first_event_at) { - error!( - "Failed to update first_event_at in stream info for stream {:?}: {err:?}", - stream_name - ); - } - - Some(first_event_at.to_string()) -} diff --git a/src/handlers/http/modal/utils/rbac_utils.rs b/src/handlers/http/modal/utils/rbac_utils.rs index fb8d2e276..b7108121a 100644 --- a/src/handlers/http/modal/utils/rbac_utils.rs +++ b/src/handlers/http/modal/utils/rbac_utils.rs @@ -17,13 +17,13 @@ */ use crate::{ - option::CONFIG, + parseable::PARSEABLE, storage::{self, ObjectStorageError, StorageMetadata}, }; pub async fn get_metadata() -> Result { - let metadata = CONFIG - .storage() + let metadata = PARSEABLE + .storage .get_object_store() .get_metadata() .await? diff --git a/src/handlers/http/oidc.rs b/src/handlers/http/oidc.rs index ba25f7c5b..286f5f29c 100644 --- a/src/handlers/http/oidc.rs +++ b/src/handlers/http/oidc.rs @@ -34,7 +34,7 @@ use url::Url; use crate::{ handlers::{COOKIE_AGE_DAYS, OIDC_SCOPE, SESSION_COOKIE_NAME, USER_COOKIE_NAME}, oidc::{Claims, DiscoveredClient}, - option::CONFIG, + parseable::PARSEABLE, rbac::{ self, map::{SessionKey, DEFAULT_ROLE}, @@ -197,7 +197,7 @@ pub async fn reply_login( let redirect_url = login_query .state .clone() - .unwrap_or_else(|| CONFIG.options.address.to_string()); + .unwrap_or_else(|| PARSEABLE.options.address.to_string()); Ok(redirect_to_client( &redirect_url, @@ -351,8 +351,8 @@ async fn update_user_if_changed( } async fn get_metadata() -> Result { - let metadata = CONFIG - .storage() + let metadata = PARSEABLE + .storage .get_object_store() .get_metadata() .await? diff --git a/src/handlers/http/query.rs b/src/handlers/http/query.rs index 5cfc791ad..9e5d405b0 100644 --- a/src/handlers/http/query.rs +++ b/src/handlers/http/query.rs @@ -35,11 +35,11 @@ use tracing::error; use crate::event::error::EventError; use crate::handlers::http::fetch_schema; -use crate::metadata::STREAM_INFO; use crate::event::commit_schema; use crate::metrics::QUERY_EXECUTE_TIME; -use crate::option::{Mode, CONFIG}; +use crate::option::Mode; +use crate::parseable::PARSEABLE; use crate::query::error::ExecuteError; use crate::query::{CountsRequest, CountsResponse, Query as LogicalQuery}; use crate::query::{TableScanVisitor, QUERY_SESSION}; @@ -51,8 +51,6 @@ use crate::utils::actix::extract_session_key_from_req; use crate::utils::time::{TimeParseError, TimeRange}; use crate::utils::user_auth_for_query; -use super::modal::utils::logstream_utils::create_stream_and_schema_from_storage; - /// Query Request through http endpoint. #[derive(Debug, Deserialize, Serialize, Clone)] #[serde(rename_all = "camelCase")] @@ -170,7 +168,7 @@ pub async fn get_counts( } pub async fn update_schema_when_distributed(tables: &Vec) -> Result<(), QueryError> { - if CONFIG.options.mode == Mode::Query { + if PARSEABLE.options.mode == Mode::Query { for table in tables { if let Ok(new_schema) = fetch_schema(table).await { // commit schema merges the schema internally and updates the schema in storage. @@ -188,12 +186,14 @@ pub async fn update_schema_when_distributed(tables: &Vec) -> Result<(), /// get list of streams from memory and storage /// create streams for memory from storage if they do not exist pub async fn create_streams_for_querier() { - let querier_streams = STREAM_INFO.list_streams(); - let store = CONFIG.storage().get_object_store(); + let querier_streams = PARSEABLE.streams.list(); + let store = PARSEABLE.storage.get_object_store(); let storage_streams = store.list_streams().await.unwrap(); for stream_name in storage_streams { if !querier_streams.contains(&stream_name) { - let _ = create_stream_and_schema_from_storage(&stream_name).await; + let _ = PARSEABLE + .create_stream_and_schema_from_storage(&stream_name) + .await; } } } diff --git a/src/handlers/http/role.rs b/src/handlers/http/role.rs index 711eade9b..f8f2c1c4f 100644 --- a/src/handlers/http/role.rs +++ b/src/handlers/http/role.rs @@ -24,7 +24,7 @@ use actix_web::{ use http::StatusCode; use crate::{ - option::CONFIG, + parseable::PARSEABLE, rbac::{ map::{mut_roles, DEFAULT_ROLE}, role::model::DefaultPrivilege, @@ -102,8 +102,8 @@ pub async fn get_default() -> Result { } async fn get_metadata() -> Result { - let metadata = CONFIG - .storage() + let metadata = PARSEABLE + .storage .get_object_store() .get_metadata() .await? diff --git a/src/handlers/http/users/dashboards.rs b/src/handlers/http/users/dashboards.rs index f95de4559..32619f0f0 100644 --- a/src/handlers/http/users/dashboards.rs +++ b/src/handlers/http/users/dashboards.rs @@ -18,7 +18,7 @@ use crate::{ handlers::http::rbac::RBACError, - option::CONFIG, + parseable::PARSEABLE, storage::{object_storage::dashboard_path, ObjectStorageError}, users::dashboards::{Dashboard, CURRENT_DASHBOARD_VERSION, DASHBOARDS}, utils::{get_hash, get_user_from_request}, @@ -81,7 +81,7 @@ pub async fn post( let path = dashboard_path(&user_id, &format!("{}.json", dashboard_id)); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let dashboard_bytes = serde_json::to_vec(&dashboard)?; store .put_object(&path, Bytes::from(dashboard_bytes)) @@ -114,7 +114,7 @@ pub async fn update( let path = dashboard_path(&user_id, &format!("{}.json", dashboard_id)); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let dashboard_bytes = serde_json::to_vec(&dashboard)?; store .put_object(&path, Bytes::from(dashboard_bytes)) @@ -134,7 +134,7 @@ pub async fn delete( return Err(DashboardError::Metadata("Dashboard does not exist")); } let path = dashboard_path(&user_id, &format!("{}.json", dashboard_id)); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); store.delete_object(&path).await?; DASHBOARDS.delete_dashboard(&dashboard_id); diff --git a/src/handlers/http/users/filters.rs b/src/handlers/http/users/filters.rs index c3e2ed905..7ea3c8a09 100644 --- a/src/handlers/http/users/filters.rs +++ b/src/handlers/http/users/filters.rs @@ -18,7 +18,7 @@ use crate::{ handlers::http::rbac::RBACError, - option::CONFIG, + parseable::PARSEABLE, storage::{object_storage::filter_path, ObjectStorageError}, users::filters::{Filter, CURRENT_FILTER_VERSION, FILTERS}, utils::{get_hash, get_user_from_request}, @@ -71,7 +71,7 @@ pub async fn post( &format!("{}.json", filter_id), ); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let filter_bytes = serde_json::to_vec(&filter)?; store.put_object(&path, Bytes::from(filter_bytes)).await?; @@ -100,7 +100,7 @@ pub async fn update( &format!("{}.json", filter_id), ); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let filter_bytes = serde_json::to_vec(&filter)?; store.put_object(&path, Bytes::from(filter_bytes)).await?; @@ -123,7 +123,7 @@ pub async fn delete( &filter.stream_name, &format!("{}.json", filter_id), ); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); store.delete_object(&path).await?; FILTERS.delete_filter(&filter_id); diff --git a/src/handlers/livetail.rs b/src/handlers/livetail.rs index ff968b811..bfaa7205e 100644 --- a/src/handlers/livetail.rs +++ b/src/handlers/livetail.rs @@ -40,8 +40,7 @@ use tower_http::cors::CorsLayer; use tracing::{info, warn}; use crate::livetail::{Message, LIVETAIL}; -use crate::metadata::STREAM_INFO; -use crate::option::CONFIG; +use crate::parseable::PARSEABLE; use crate::rbac::map::SessionKey; use crate::rbac::{self, Users}; use crate::utils; @@ -116,9 +115,10 @@ impl FlightService for FlightServiceImpl { } } - let schema = STREAM_INFO - .schema(stream) - .map_err(|err| Status::failed_precondition(err.to_string()))?; + let schema = PARSEABLE + .get_stream(stream) + .map_err(|err| Status::failed_precondition(err.to_string()))? + .get_schema(); let rx = LIVETAIL.new_pipe( Alphanumeric.sample_string(&mut rand::thread_rng(), 32), @@ -172,12 +172,12 @@ impl FlightService for FlightServiceImpl { } pub fn server() -> impl Future>> + Send { - let mut addr: SocketAddr = CONFIG + let mut addr: SocketAddr = PARSEABLE .options .address .parse() .expect("valid socket address"); - addr.set_port(CONFIG.options.grpc_port); + addr.set_port(PARSEABLE.options.grpc_port); let service = FlightServiceImpl {}; @@ -185,7 +185,10 @@ pub fn server() -> impl Future { match (std::fs::read_to_string(cert), std::fs::read_to_string(key)) { (Ok(cert_file), Ok(key_file)) => { diff --git a/src/handlers/mod.rs b/src/handlers/mod.rs index 2f0ae762f..f584dcd97 100644 --- a/src/handlers/mod.rs +++ b/src/handlers/mod.rs @@ -28,7 +28,7 @@ const CUSTOM_PARTITION_KEY: &str = "x-p-custom-partition"; const STATIC_SCHEMA_FLAG: &str = "x-p-static-schema-flag"; const AUTHORIZATION_KEY: &str = "authorization"; const UPDATE_STREAM_KEY: &str = "x-p-update-stream"; -const STREAM_TYPE_KEY: &str = "x-p-stream-type"; +pub const STREAM_TYPE_KEY: &str = "x-p-stream-type"; const OIDC_SCOPE: &str = "openid profile email"; const COOKIE_AGE_DAYS: usize = 7; const SESSION_COOKIE_NAME: &str = "session"; diff --git a/src/hottier.rs b/src/hottier.rs index 329048749..a8128641d 100644 --- a/src/hottier.rs +++ b/src/hottier.rs @@ -26,8 +26,7 @@ use std::{ use crate::{ catalog::manifest::{File, Manifest}, handlers::http::cluster::INTERNAL_STREAM_NAME, - metadata::{error::stream_info::MetadataError, STREAM_INFO}, - option::CONFIG, + parseable::PARSEABLE, storage::{ObjectStorage, ObjectStorageError}, utils::{extract_datetime, human_size::bytes_to_human_size}, validator::error::HotTierValidationError, @@ -84,7 +83,7 @@ impl HotTierManager { pub fn global() -> Option<&'static HotTierManager> { static INSTANCE: OnceCell = OnceCell::new(); - CONFIG + PARSEABLE .options .hot_tier_storage_path .as_ref() @@ -98,7 +97,7 @@ impl HotTierManager { ) -> 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() { + for stream in PARSEABLE.streams.list() { 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 += &stream_hot_tier.size; @@ -142,7 +141,7 @@ impl HotTierManager { let (total_hot_tier_size, total_hot_tier_used_size) = self.get_hot_tiers_size(stream).await?; - let disk_threshold = (CONFIG.options.max_disk_usage * total_space as f64) / 100.0; + let disk_threshold = (PARSEABLE.options.max_disk_usage * total_space as f64) / 100.0; let max_allowed_hot_tier_size = disk_threshold - total_hot_tier_size as f64 - (used_space as f64 @@ -246,7 +245,7 @@ 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 mut sync_hot_tier_tasks = FuturesUnordered::new(); - for stream in STREAM_INFO.list_streams() { + for stream in PARSEABLE.streams.list() { if self.check_stream_hot_tier_exists(&stream) { sync_hot_tier_tasks.push(self.process_stream(stream)); } @@ -267,7 +266,7 @@ impl HotTierManager { let stream_hot_tier = self.get_hot_tier(&stream).await?; let mut parquet_file_size = stream_hot_tier.used_size; - let object_store = CONFIG.storage().get_object_store(); + let object_store = PARSEABLE.storage.get_object_store(); let mut s3_manifest_file_list = object_store.list_manifest_files(&stream).await?; self.process_manifest( &stream, @@ -375,8 +374,8 @@ impl HotTierManager { 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() + let parquet_data = PARSEABLE + .storage .get_object_store() .get_object(&parquet_file_path) .await?; @@ -632,7 +631,7 @@ impl HotTierManager { } if ((used_space + size_to_download) as f64 * 100.0 / total_space as f64) - > CONFIG.options.max_disk_usage + > PARSEABLE.options.max_disk_usage { return Ok(false); } @@ -784,8 +783,6 @@ pub enum HotTierError { #[error("{0}")] ParquetError(#[from] ParquetError), #[error("{0}")] - MetadataError(#[from] MetadataError), - #[error("{0}")] HotTierValidationError(#[from] HotTierValidationError), #[error("{0}")] Anyhow(#[from] anyhow::Error), diff --git a/src/lib.rs b/src/lib.rs index 061595abe..d26e37974 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -36,10 +36,10 @@ pub mod migration; mod oidc; pub mod option; pub mod otel; +pub mod parseable; mod query; pub mod rbac; mod response; -mod staging; mod static_schema; mod stats; pub mod storage; @@ -56,6 +56,9 @@ pub use handlers::http::modal::{ use once_cell::sync::Lazy; use reqwest::{Client, ClientBuilder}; +// It is very unlikely that panic will occur when dealing with locks. +pub const LOCK_EXPECT: &str = "Thread shouldn't panic while holding a lock"; + pub const STORAGE_CONVERSION_INTERVAL: u64 = 60; pub const STORAGE_UPLOAD_INTERVAL: u64 = 30; diff --git a/src/main.rs b/src/main.rs index 492121490..e652ca8c2 100644 --- a/src/main.rs +++ b/src/main.rs @@ -18,9 +18,8 @@ #[cfg(feature = "kafka")] use parseable::connectors; use parseable::{ - banner, metrics, - option::{Mode, CONFIG}, - rbac, storage, IngestServer, ParseableServer, QueryServer, Server, + banner, metrics, option::Mode, parseable::PARSEABLE, rbac, storage, IngestServer, + ParseableServer, QueryServer, Server, }; use tokio::signal::ctrl_c; use tokio::sync::oneshot; @@ -35,7 +34,7 @@ async fn main() -> anyhow::Result<()> { init_logger(); // these are empty ptrs so mem footprint should be minimal - let server: Box = match CONFIG.options.mode { + let server: Box = match &PARSEABLE.options.mode { Mode::Query => Box::new(QueryServer), Mode::Ingest => Box::new(IngestServer), Mode::All => Box::new(Server), @@ -44,7 +43,7 @@ async fn main() -> anyhow::Result<()> { // load metadata from persistence let parseable_json = server.load_metadata().await?; let metadata = storage::resolve_parseable_metadata(&parseable_json).await?; - banner::print(&CONFIG, &metadata).await; + banner::print(&PARSEABLE, &metadata).await; // initialize the rbac map rbac::map::init(&metadata); // keep metadata info in mem diff --git a/src/metadata.rs b/src/metadata.rs index e98d7e52a..a29fdfee2 100644 --- a/src/metadata.rs +++ b/src/metadata.rs @@ -16,35 +16,49 @@ * */ -use arrow_schema::{DataType, Field, Fields, Schema, TimeUnit}; +use arrow_schema::{DataType, Field, Schema, TimeUnit}; use chrono::{Local, NaiveDateTime}; -use itertools::Itertools; -use once_cell::sync::Lazy; use serde::{Deserialize, Serialize}; -use serde_json::Value; use std::collections::HashMap; use std::num::NonZeroU32; -use std::sync::{Arc, RwLock}; +use std::sync::Arc; -use self::error::stream_info::{LoadError, MetadataError}; use crate::catalog::snapshot::ManifestItem; use crate::event::format::LogSource; use crate::metrics::{ - fetch_stats_from_storage, EVENTS_INGESTED, EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE, - EVENTS_INGESTED_SIZE_DATE, EVENTS_STORAGE_SIZE_DATE, LIFETIME_EVENTS_INGESTED, - LIFETIME_EVENTS_INGESTED_SIZE, + EVENTS_INGESTED, EVENTS_INGESTED_DATE, EVENTS_INGESTED_SIZE, EVENTS_INGESTED_SIZE_DATE, + EVENTS_STORAGE_SIZE_DATE, LIFETIME_EVENTS_INGESTED, LIFETIME_EVENTS_INGESTED_SIZE, }; -use crate::staging::STAGING; use crate::storage::retention::Retention; -use crate::storage::{ObjectStorage, ObjectStoreFormat, StreamType}; -use derive_more::{Deref, DerefMut}; +use crate::storage::StreamType; -// TODO: make return type be of 'static lifetime instead of cloning -// A read-write lock to allow multiple reads while and isolated write -pub static STREAM_INFO: Lazy = Lazy::new(StreamInfo::default); - -#[derive(Debug, Deref, DerefMut, Default)] -pub struct StreamInfo(RwLock>); +pub fn update_stats( + stream_name: &str, + origin: &'static str, + size: u64, + num_rows: usize, + parsed_timestamp: NaiveDateTime, +) { + let parsed_date = parsed_timestamp.date().to_string(); + EVENTS_INGESTED + .with_label_values(&[stream_name, origin]) + .add(num_rows as i64); + EVENTS_INGESTED_DATE + .with_label_values(&[stream_name, origin, parsed_date.as_str()]) + .add(num_rows as i64); + EVENTS_INGESTED_SIZE + .with_label_values(&[stream_name, origin]) + .add(size as i64); + EVENTS_INGESTED_SIZE_DATE + .with_label_values(&[stream_name, origin, parsed_date.as_str()]) + .add(size as i64); + LIFETIME_EVENTS_INGESTED + .with_label_values(&[stream_name, origin]) + .add(num_rows as i64); + LIFETIME_EVENTS_INGESTED_SIZE + .with_label_values(&[stream_name, origin]) + .add(size as i64); +} /// In order to support backward compatability with streams created before v1.6.4, /// we will consider past versions of stream schema to be v0. Streams created with @@ -76,222 +90,20 @@ pub struct LogStreamMetadata { pub log_source: LogSource, } -// It is very unlikely that panic will occur when dealing with metadata. -pub const LOCK_EXPECT: &str = "no method in metadata should panic while holding a lock"; - -// STREAM_INFO should be updated -// 1. During server start up -// 2. When a new stream is created (make a new entry in the map) -// 3. When a stream is deleted (remove the entry from the map) -// 4. When first event is sent to stream (update the schema) -// 5. When set alert API is called (update the alert) -impl StreamInfo { - pub fn stream_exists(&self, stream_name: &str) -> bool { - let map = self.read().expect(LOCK_EXPECT); - map.contains_key(stream_name) - } - - pub fn get_first_event(&self, stream_name: &str) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.first_event_at.clone()) - } - - pub fn get_time_partition(&self, stream_name: &str) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.time_partition.clone()) - } - - pub fn get_time_partition_limit( - &self, - stream_name: &str, - ) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.time_partition_limit) - } - - pub fn get_custom_partition(&self, stream_name: &str) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.custom_partition.clone()) - } - - pub fn get_static_schema_flag(&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.static_schema_flag) - } - - pub fn get_retention(&self, stream_name: &str) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - map.get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.retention.clone()) - } - - pub fn get_schema_version(&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.schema_version) - } - - pub fn schema(&self, stream_name: &str) -> Result, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - let schema = map - .get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| &metadata.schema)?; - - // sort fields on read from hashmap as order of fields can differ. - // This provides a stable output order if schema is same between calls to this function - let fields: Fields = schema - .values() - .sorted_by_key(|field| field.name()) - .cloned() - .collect(); - - let schema = Schema::new(fields); - - Ok(Arc::new(schema)) - } - - #[allow(dead_code)] - pub fn schema_raw( - &self, - stream_name: &str, - ) -> Result>, MetadataError> { - let map = self.read().expect(LOCK_EXPECT); - - let schema = map - .get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.schema.clone())?; - - Ok(schema) - } - - pub fn set_retention( - &self, - stream_name: &str, - retention: Retention, - ) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - metadata.retention = Some(retention); - }) - } - - pub fn set_first_event_at( - &self, - stream_name: &str, - first_event_at: &str, - ) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - metadata.first_event_at = Some(first_event_at.to_owned()); - }) - } - - /// Removes the `first_event_at` timestamp for the specified stream from the LogStreamMetadata. - /// - /// This function is called during the retention task, when the parquet files along with the manifest files are deleted from the storage. - /// The manifest path is removed from the snapshot in the stream.json - /// and the first_event_at value in the stream.json is removed. - /// - /// # Arguments - /// - /// * `stream_name` - The name of the stream for which the `first_event_at` timestamp is to be removed. - /// - /// # Returns - /// - /// * `Result<(), MetadataError>` - Returns `Ok(())` if the `first_event_at` timestamp is successfully removed, - /// or a `MetadataError` if the stream metadata is not found. - /// - /// # Examples - /// ```ignore - /// ```rust - /// let result = metadata.remove_first_event_at("my_stream"); - /// match result { - /// Ok(()) => println!("first-event-at removed successfully"), - /// Err(e) => eprintln!("Error removing first-event-at from STREAM_INFO: {}", e), - /// } - /// ``` - pub fn reset_first_event_at(&self, stream_name: &str) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - metadata.first_event_at.take(); - }) - } - - pub fn update_time_partition_limit( - &self, - stream_name: &str, - time_partition_limit: NonZeroU32, - ) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - metadata.time_partition_limit = Some(time_partition_limit); - }) - } - - pub fn update_custom_partition( - &self, - stream_name: &str, - custom_partition: String, - ) -> Result<(), MetadataError> { - let mut map = self.write().expect(LOCK_EXPECT); - map.get_mut(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| { - if custom_partition.is_empty() { - metadata.custom_partition = None; - return; - } - metadata.custom_partition = Some(custom_partition); - }) - } - - 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 = enable; - Ok(()) - } - +impl LogStreamMetadata { #[allow(clippy::too_many_arguments)] - pub fn add_stream( - &self, - stream_name: String, + pub fn new( created_at: String, time_partition: String, time_partition_limit: Option, - custom_partition: String, + custom_partition: Option, static_schema_flag: bool, static_schema: HashMap>, stream_type: StreamType, schema_version: SchemaVersion, log_source: LogSource, - ) { - let mut map = self.write().expect(LOCK_EXPECT); - let metadata = LogStreamMetadata { + ) -> Self { + LogStreamMetadata { created_at: if created_at.is_empty() { Local::now().to_rfc3339() } else { @@ -303,11 +115,7 @@ impl StreamInfo { Some(time_partition) }, time_partition_limit, - custom_partition: if custom_partition.is_empty() { - None - } else { - Some(custom_partition) - }, + custom_partition, static_schema_flag, schema: if static_schema.is_empty() { HashMap::new() @@ -318,68 +126,7 @@ impl StreamInfo { schema_version, log_source, ..Default::default() - }; - map.insert(stream_name, metadata); - } - - pub fn delete_stream(&self, stream_name: &str) { - let mut map = self.write().expect(LOCK_EXPECT); - map.remove(stream_name); - } - - pub fn list_streams(&self) -> Vec { - self.read() - .expect(LOCK_EXPECT) - .keys() - .map(String::clone) - .collect() - } - - pub fn list_internal_streams(&self) -> Vec { - self.read() - .expect(LOCK_EXPECT) - .iter() - .filter(|(_, v)| v.stream_type == StreamType::Internal) - .map(|(k, _)| k.clone()) - .collect() - } - - pub fn stream_type(&self, stream_name: &str) -> Result { - self.read() - .expect(LOCK_EXPECT) - .get(stream_name) - .ok_or(MetadataError::StreamMetaNotFound(stream_name.to_string())) - .map(|metadata| metadata.stream_type) - } - - pub fn update_stats( - &self, - stream_name: &str, - origin: &'static str, - size: u64, - num_rows: usize, - parsed_timestamp: NaiveDateTime, - ) -> Result<(), MetadataError> { - let parsed_date = parsed_timestamp.date().to_string(); - EVENTS_INGESTED - .with_label_values(&[stream_name, origin]) - .add(num_rows as i64); - EVENTS_INGESTED_DATE - .with_label_values(&[stream_name, origin, parsed_date.as_str()]) - .add(num_rows as i64); - EVENTS_INGESTED_SIZE - .with_label_values(&[stream_name, origin]) - .add(size as i64); - EVENTS_INGESTED_SIZE_DATE - .with_label_values(&[stream_name, origin, parsed_date.as_str()]) - .add(size as i64); - LIFETIME_EVENTS_INGESTED - .with_label_values(&[stream_name, origin]) - .add(num_rows as i64); - LIFETIME_EVENTS_INGESTED_SIZE - .with_label_values(&[stream_name, origin]) - .add(size as i64); - Ok(()) + } } } @@ -389,12 +136,9 @@ impl StreamInfo { /// required only when migrating from version 1.2.0 and below /// this function will be removed in the future pub async fn update_data_type_time_partition( - storage: &(impl ObjectStorage + ?Sized), - stream_name: &str, - schema: Schema, + schema: &mut Schema, time_partition: Option<&String>, -) -> anyhow::Result { - let mut schema = schema.clone(); +) -> anyhow::Result<()> { if let Some(time_partition) = time_partition { if let Ok(time_partition_field) = schema.field_with_name(time_partition) { if time_partition_field.data_type() != &DataType::Timestamp(TimeUnit::Millisecond, None) @@ -411,81 +155,15 @@ pub async fn update_data_type_time_partition( true, )); fields.push(time_partition_field); - schema = Schema::new(fields); - storage.put_schema(stream_name, &schema).await?; + *schema = Schema::new(fields); } } } - Ok(schema) -} - -pub async fn load_stream_metadata_on_server_start( - storage: &(impl ObjectStorage + ?Sized), - stream_name: &str, - schema: Schema, - stream_metadata_value: Value, -) -> Result<(), LoadError> { - let ObjectStoreFormat { - schema_version, - created_at, - first_event_at, - retention, - snapshot, - stats, - time_partition, - time_partition_limit, - custom_partition, - static_schema_flag, - hot_tier_enabled, - stream_type, - log_source, - .. - } = if !stream_metadata_value.is_null() { - serde_json::from_slice(&serde_json::to_vec(&stream_metadata_value).unwrap()).unwrap() - } else { - ObjectStoreFormat::default() - }; - let schema = - update_data_type_time_partition(storage, stream_name, schema, time_partition.as_ref()) - .await?; - storage.put_schema(stream_name, &schema).await?; - //load stats from storage - fetch_stats_from_storage(stream_name, stats).await; - load_daily_metrics(&snapshot.manifest_list, stream_name); - - let schema = STAGING - .get_or_create_stream(stream_name) - .updated_schema(schema); - let schema = HashMap::from_iter( - schema - .fields - .iter() - .map(|v| (v.name().to_owned(), v.clone())), - ); - - let metadata = LogStreamMetadata { - schema_version, - schema, - retention, - created_at, - first_event_at, - time_partition, - time_partition_limit: time_partition_limit.and_then(|limit| limit.parse().ok()), - custom_partition, - static_schema_flag, - hot_tier_enabled, - stream_type, - log_source, - }; - - let mut map = STREAM_INFO.write().expect(LOCK_EXPECT); - - map.insert(stream_name.to_string(), metadata); Ok(()) } -fn load_daily_metrics(manifests: &Vec, stream_name: &str) { +pub fn load_daily_metrics(manifests: &Vec, stream_name: &str) { for manifest in manifests { let manifest_date = manifest.time_lower_bound.date_naive().to_string(); let events_ingested = manifest.events_ingested; @@ -502,25 +180,3 @@ fn load_daily_metrics(manifests: &Vec, stream_name: &str) { .set(storage_size as i64); } } - -pub mod error { - pub mod stream_info { - use crate::storage::ObjectStorageError; - - #[derive(Debug, thiserror::Error)] - pub enum MetadataError { - #[error("Metadata for stream {0} not found. Please create the stream and try again")] - StreamMetaNotFound(String), - #[error("Metadata Error: {0}")] - StandaloneWithDistributed(String), - } - - #[derive(Debug, thiserror::Error)] - pub enum LoadError { - #[error("Error while loading from object storage: {0}")] - ObjectStorage(#[from] ObjectStorageError), - #[error(" Error: {0}")] - Anyhow(#[from] anyhow::Error), - } - } -} diff --git a/src/migration/metadata_migration.rs b/src/migration/metadata_migration.rs index 5de34a690..c42ebe94c 100644 --- a/src/migration/metadata_migration.rs +++ b/src/migration/metadata_migration.rs @@ -16,14 +16,10 @@ * */ -use bytes::Bytes; use rand::distributions::DistString; use serde_json::{json, Map, Value as JsonValue}; -use crate::{ - handlers::http::modal::IngestorMetadata, option::CONFIG, - storage::object_storage::ingestor_metadata_path, -}; +use crate::parseable::PARSEABLE; /* v1 @@ -46,7 +42,7 @@ pub fn v1_v3(mut storage_metadata: JsonValue) -> JsonValue { metadata.insert("users".to_string(), JsonValue::Array(vec![])); metadata.insert("streams".to_string(), JsonValue::Array(vec![])); metadata.insert("roles".to_string(), JsonValue::Array(vec![])); - metadata.insert("server_mode".to_string(), json!(CONFIG.options.mode)); + metadata.insert("server_mode".to_string(), json!(PARSEABLE.options.mode)); storage_metadata } @@ -107,7 +103,7 @@ pub fn v2_v3(mut storage_metadata: JsonValue) -> JsonValue { "roles".to_string(), JsonValue::Object(Map::from_iter(privileges_map)), ); - metadata.insert("server_mode".to_string(), json!(CONFIG.options.mode)); + metadata.insert("server_mode".to_string(), json!(PARSEABLE.options.mode)); storage_metadata } @@ -118,7 +114,7 @@ pub fn v3_v4(mut storage_metadata: JsonValue) -> JsonValue { let sm = metadata.get("server_mode"); if sm.is_none() || sm.unwrap().as_str().unwrap() == "All" { - metadata.insert("server_mode".to_string(), json!(CONFIG.options.mode)); + metadata.insert("server_mode".to_string(), json!(PARSEABLE.options.mode)); } let roles = metadata.get_mut("roles").unwrap().as_object_mut().unwrap(); @@ -145,17 +141,17 @@ pub fn v4_v5(mut storage_metadata: JsonValue) -> JsonValue { match metadata.get("server_mode") { None => { - metadata.insert("server_mode".to_string(), json!(CONFIG.options.mode)); + metadata.insert("server_mode".to_string(), json!(PARSEABLE.options.mode)); } Some(JsonValue::String(mode)) => match mode.as_str() { "Query" => { metadata.insert( "querier_endpoint".to_string(), - JsonValue::String(CONFIG.options.address.clone()), + JsonValue::String(PARSEABLE.options.address.clone()), ); } "All" => { - metadata.insert("server_mode".to_string(), json!(CONFIG.options.mode)); + metadata.insert("server_mode".to_string(), json!(PARSEABLE.options.mode)); } _ => (), }, @@ -172,37 +168,3 @@ pub fn remove_querier_metadata(mut storage_metadata: JsonValue) -> JsonValue { metadata.remove("querier_auth_token"); storage_metadata } - -pub async fn migrate_ingester_metadata() -> anyhow::Result> { - let imp = ingestor_metadata_path(None); - let bytes = match CONFIG.storage().get_object_store().get_object(&imp).await { - Ok(bytes) => bytes, - Err(_) => { - return Ok(None); - } - }; - let mut json = serde_json::from_slice::(&bytes)?; - let meta = json - .as_object_mut() - .ok_or_else(|| anyhow::anyhow!("Unable to parse Ingester Metadata"))?; - let fp = meta.get("flight_port"); - - if fp.is_none() { - meta.insert( - "flight_port".to_owned(), - JsonValue::String(CONFIG.options.flight_port.to_string()), - ); - } - let bytes = Bytes::from(serde_json::to_vec(&json)?); - - let resource: IngestorMetadata = serde_json::from_value(json)?; - resource.put_on_disk(CONFIG.staging_dir())?; - - CONFIG - .storage() - .get_object_store() - .put_object(&imp, bytes) - .await?; - - Ok(Some(resource)) -} diff --git a/src/migration/mod.rs b/src/migration/mod.rs index 38447b5d9..da8b795a1 100644 --- a/src/migration/mod.rs +++ b/src/migration/mod.rs @@ -21,31 +21,33 @@ pub mod metadata_migration; mod schema_migration; mod stream_metadata_migration; -use std::{fs::OpenOptions, sync::Arc}; +use std::{collections::HashMap, fs::OpenOptions, sync::Arc}; -use crate::{ - metadata::load_stream_metadata_on_server_start, - option::{Config, Mode, CONFIG}, - storage::{ - object_storage::{parseable_json_path, schema_path, stream_json_path}, - ObjectStorage, ObjectStorageError, PARSEABLE_METADATA_FILE_NAME, PARSEABLE_ROOT_DIRECTORY, - STREAM_ROOT_DIRECTORY, - }, -}; use arrow_schema::Schema; use bytes::Bytes; use relative_path::RelativePathBuf; use serde::Serialize; use serde_json::Value; -use tracing::error; + +use crate::{ + metadata::{load_daily_metrics, update_data_type_time_partition, LogStreamMetadata}, + metrics::fetch_stats_from_storage, + option::Mode, + parseable::{Parseable, PARSEABLE}, + storage::{ + object_storage::{parseable_json_path, schema_path, stream_json_path}, + ObjectStorage, ObjectStorageError, ObjectStoreFormat, PARSEABLE_METADATA_FILE_NAME, + PARSEABLE_ROOT_DIRECTORY, STREAM_ROOT_DIRECTORY, + }, +}; /// Migrate the metdata from v1 or v2 to v3 /// This is a one time migration pub async fn run_metadata_migration( - config: &Config, + config: &Parseable, parseable_json: &Option, ) -> anyhow::Result<()> { - let object_store = config.storage().get_object_store(); + let object_store = config.storage.get_object_store(); let mut storage_metadata: Option = None; if parseable_json.is_some() { storage_metadata = serde_json::from_slice(parseable_json.as_ref().unwrap()) @@ -130,16 +132,25 @@ pub async fn run_metadata_migration( } /// run the migration for all streams -pub async fn run_migration(config: &Config) -> anyhow::Result<()> { - let storage = config.storage().get_object_store(); +pub async fn run_migration(config: &Parseable) -> anyhow::Result<()> { + let storage = config.storage.get_object_store(); for stream_name in storage.list_streams().await? { - migration_stream(&stream_name, &*storage).await?; + let Some(metadata) = migration_stream(&stream_name, &*storage).await? else { + continue; + }; + config + .get_or_create_stream(&stream_name) + .set_metadata(metadata) + .await; } Ok(()) } -async fn migration_stream(stream: &str, storage: &dyn ObjectStorage) -> anyhow::Result<()> { +async fn migration_stream( + stream: &str, + storage: &dyn ObjectStorage, +) -> anyhow::Result> { let mut arrow_schema: Schema = Schema::empty(); //check if schema exists for the node @@ -186,8 +197,8 @@ async fn migration_stream(stream: &str, storage: &dyn ObjectStorage) -> anyhow:: let mut stream_meta_found = true; if stream_metadata.is_empty() { - if CONFIG.options.mode != Mode::Ingest { - return Ok(()); + if PARSEABLE.options.mode != Mode::Ingest { + return Ok(None); } stream_meta_found = false; } @@ -251,15 +262,59 @@ async fn migration_stream(stream: &str, storage: &dyn ObjectStorage) -> anyhow:: arrow_schema = serde_json::from_slice(&schema)?; } - if let Err(err) = - load_stream_metadata_on_server_start(storage, stream, arrow_schema, stream_metadata_value) - .await - { - error!("could not populate local metadata. {:?}", err); - return Err(err.into()); - } + // Setup logstream meta on startup + let ObjectStoreFormat { + schema_version, + created_at, + first_event_at, + retention, + snapshot, + stats, + time_partition, + time_partition_limit, + custom_partition, + static_schema_flag, + hot_tier_enabled, + stream_type, + log_source, + .. + } = serde_json::from_value(stream_metadata_value).unwrap_or_default(); + let storage = PARSEABLE.storage.get_object_store(); + + // update the schema and store it back + // NOTE: write could be saved, but the cost is cheap, given the low possibilities of being called multiple times + update_data_type_time_partition(&mut arrow_schema, time_partition.as_ref()).await?; + storage.put_schema(stream, &arrow_schema).await?; + //load stats from storage + fetch_stats_from_storage(stream, stats).await; + load_daily_metrics(&snapshot.manifest_list, stream); + + let schema = PARSEABLE + .get_or_create_stream(stream) + .updated_schema(arrow_schema); + let schema = HashMap::from_iter( + schema + .fields + .iter() + .map(|v| (v.name().to_owned(), v.clone())), + ); + + let metadata = LogStreamMetadata { + schema_version, + schema, + retention, + created_at, + first_event_at, + time_partition, + time_partition_limit: time_partition_limit.and_then(|limit| limit.parse().ok()), + custom_partition, + static_schema_flag, + hot_tier_enabled, + stream_type, + log_source, + }; - Ok(()) + Ok(Some(metadata)) } #[inline(always)] @@ -269,7 +324,7 @@ pub fn to_bytes(any: &(impl ?Sized + Serialize)) -> Bytes { .expect("serialize cannot fail") } -pub fn get_staging_metadata(config: &Config) -> anyhow::Result> { +pub fn get_staging_metadata(config: &Parseable) -> anyhow::Result> { let path = RelativePathBuf::from(PARSEABLE_METADATA_FILE_NAME).to_path(config.staging_dir()); let bytes = match std::fs::read(path) { Ok(bytes) => bytes, @@ -292,7 +347,10 @@ pub async fn put_remote_metadata( Ok(storage.put_object(&path, metadata).await?) } -pub fn put_staging_metadata(config: &Config, metadata: &serde_json::Value) -> anyhow::Result<()> { +pub fn put_staging_metadata( + config: &Parseable, + metadata: &serde_json::Value, +) -> anyhow::Result<()> { let path = config.staging_dir().join(".parseable.json"); let mut file = OpenOptions::new() .create(true) @@ -303,8 +361,8 @@ pub fn put_staging_metadata(config: &Config, metadata: &serde_json::Value) -> an Ok(()) } -pub async fn run_file_migration(config: &Config) -> anyhow::Result<()> { - let object_store = config.storage().get_object_store(); +pub async fn run_file_migration(config: &Parseable) -> anyhow::Result<()> { + let object_store = config.storage.get_object_store(); let old_meta_file_path = RelativePathBuf::from(PARSEABLE_METADATA_FILE_NAME); diff --git a/src/option.rs b/src/option.rs index 11aea57a4..9e25b1917 100644 --- a/src/option.rs +++ b/src/option.rs @@ -15,152 +15,8 @@ * along with this program. If not, see . * */ - -use crate::cli::{Cli, Options, StorageOptions, DEFAULT_PASSWORD, DEFAULT_USERNAME}; -#[cfg(feature = "kafka")] -use crate::connectors::kafka::config::KafkaConfig; -use crate::storage::object_storage::parseable_json_path; -use crate::storage::{ObjectStorageError, ObjectStorageProvider}; -use bytes::Bytes; -use clap::error::ErrorKind; -use clap::Parser; -use once_cell::sync::Lazy; use parquet::basic::{BrotliLevel, GzipLevel, ZstdLevel}; use serde::{Deserialize, Serialize}; -use std::path::PathBuf; -use std::sync::Arc; - -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())); - -#[derive(Debug)] -pub struct Config { - pub options: Options, - storage: Arc, - pub storage_name: &'static str, - #[cfg(feature = "kafka")] - pub kafka_config: KafkaConfig, -} - -impl Config { - fn new() -> Self { - match Cli::parse().storage { - StorageOptions::Local(args) => { - if args.options.local_staging_path == args.storage.root { - clap::Error::raw( - ErrorKind::ValueValidation, - "Cannot use same path for storage and staging", - ) - .exit(); - } - - if args.options.hot_tier_storage_path.is_some() { - clap::Error::raw( - ErrorKind::ValueValidation, - "Cannot use hot tier with local-store subcommand.", - ) - .exit(); - } - - Config { - options: args.options, - storage: Arc::new(args.storage), - storage_name: "drive", - #[cfg(feature = "kafka")] - kafka_config: args.kafka, - } - } - StorageOptions::S3(args) => Config { - options: args.options, - storage: Arc::new(args.storage), - storage_name: "s3", - #[cfg(feature = "kafka")] - kafka_config: args.kafka, - }, - StorageOptions::Blob(args) => Config { - options: args.options, - storage: Arc::new(args.storage), - storage_name: "blob_store", - #[cfg(feature = "kafka")] - kafka_config: args.kafka, - }, - } - } - - // validate the storage, if the proper path for staging directory is provided - // if the proper data directory is provided, or s3 bucket is provided etc - pub async fn validate_storage(&self) -> Result, ObjectStorageError> { - let obj_store = self.storage.get_object_store(); - let rel_path = parseable_json_path(); - let mut has_parseable_json = false; - let parseable_json_result = obj_store.get_object(&rel_path).await; - if parseable_json_result.is_ok() { - has_parseable_json = true; - } - - // Lists all the directories in the root of the bucket/directory - // can be a stream (if it contains .stream.json file) or not - let has_dirs = match obj_store.list_dirs().await { - Ok(dirs) => !dirs.is_empty(), - Err(_) => false, - }; - - let has_streams = obj_store.list_streams().await.is_ok(); - if !has_dirs && !has_parseable_json { - return Ok(None); - } - if has_streams { - return Ok(Some(parseable_json_result.unwrap())); - } - - if self.get_storage_mode_string() == "Local drive" { - return Err(ObjectStorageError::Custom(format!("Could not start the server because directory '{}' contains stale data, please use an empty directory, and restart the server.\n{}", self.storage.get_endpoint(), JOIN_COMMUNITY))); - } - - // S3 bucket mode - Err(ObjectStorageError::Custom(format!("Could not start the server because bucket '{}' contains stale data, please use an empty bucket and restart the server.\n{}", self.storage.get_endpoint(), JOIN_COMMUNITY))) - } - - pub fn storage(&self) -> Arc { - self.storage.clone() - } - - pub fn staging_dir(&self) -> &PathBuf { - &self.options.local_staging_path - } - - pub fn hot_tier_dir(&self) -> &Option { - &self.options.hot_tier_storage_path - } - - pub fn is_default_creds(&self) -> bool { - self.options.username == DEFAULT_USERNAME && self.options.password == DEFAULT_PASSWORD - } - - // returns the string representation of the storage mode - // drive --> Local drive - // s3 --> S3 bucket - // azure_blob --> Azure Blob Storage - pub fn get_storage_mode_string(&self) -> &str { - if self.storage_name == "drive" { - return "Local drive"; - } else if self.storage_name == "s3" { - return "S3 bucket"; - } else if self.storage_name == "blob_store" { - return "Azure Blob Storage"; - } - "Unknown" - } - - pub fn get_server_mode_string(&self) -> &str { - match self.options.mode { - Mode::Query => "Distributed (Query)", - Mode::Ingest => "Distributed (Ingest)", - Mode::All => "Standalone", - } - } -} #[derive(Debug, Default, Eq, PartialEq, Clone, Copy, Serialize, Deserialize)] pub enum Mode { @@ -170,6 +26,21 @@ pub enum Mode { All, } +#[derive(Debug, thiserror::Error)] +#[error("Starting Standalone Mode is not permitted when Distributed Mode is enabled. Please restart the server with Distributed Mode enabled.")] +pub struct StandaloneWithDistributed; + +impl Mode { + // An instance is not allowed + pub fn standalone_after_distributed(&self) -> Result<(), StandaloneWithDistributed> { + if *self == Mode::Query { + return Err(StandaloneWithDistributed); + } + + Ok(()) + } +} + #[derive(Debug, Clone, Copy, PartialEq, Eq, Default, Deserialize)] #[serde(rename_all = "lowercase")] pub enum Compression { diff --git a/src/parseable/mod.rs b/src/parseable/mod.rs new file mode 100644 index 000000000..04b754487 --- /dev/null +++ b/src/parseable/mod.rs @@ -0,0 +1,835 @@ +/* + * 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::{collections::HashMap, num::NonZeroU32, path::PathBuf, str::FromStr, sync::Arc}; + +use actix_web::http::header::HeaderMap; +use arrow_schema::{Field, Schema}; +use bytes::Bytes; +use chrono::Local; +use clap::{error::ErrorKind, Parser}; +use http::{header::CONTENT_TYPE, HeaderName, HeaderValue, StatusCode}; +use once_cell::sync::Lazy; +pub use staging::StagingError; +use streams::StreamRef; +pub use streams::{StreamNotFound, Streams}; +use tracing::error; + +#[cfg(feature = "kafka")] +use crate::connectors::kafka::config::KafkaConfig; +use crate::{ + cli::{Cli, Options, StorageOptions}, + event::format::LogSource, + handlers::{ + http::{ + cluster::{sync_streams_with_ingestors, INTERNAL_STREAM_NAME}, + ingest::PostError, + logstream::error::{CreateStreamError, StreamError}, + modal::{utils::logstream_utils::PutStreamHeaders, IngestorMetadata}, + }, + STREAM_TYPE_KEY, + }, + metadata::{LogStreamMetadata, SchemaVersion}, + option::Mode, + static_schema::{convert_static_schema_to_arrow_schema, StaticSchema}, + storage::{ + object_storage::parseable_json_path, ObjectStorageError, ObjectStorageProvider, + ObjectStoreFormat, Owner, Permisssion, StreamType, + }, + validator, +}; + +mod staging; +mod streams; + +/// Name of a Stream +/// NOTE: this used to be a struct, flattened out for simplicity +pub type LogStream = String; + +pub const JOIN_COMMUNITY: &str = + "Join us on Parseable Slack community for questions : https://logg.ing/community"; +pub const STREAM_EXISTS: &str = "Stream exists"; + +/// Shared state of the Parseable server. +pub static PARSEABLE: Lazy = Lazy::new(|| match Cli::parse().storage { + StorageOptions::Local(args) => { + if args.options.local_staging_path == args.storage.root { + clap::Error::raw( + ErrorKind::ValueValidation, + "Cannot use same path for storage and staging", + ) + .exit(); + } + + if args.options.hot_tier_storage_path.is_some() { + clap::Error::raw( + ErrorKind::ValueValidation, + "Cannot use hot tier with local-store subcommand.", + ) + .exit(); + } + + Parseable::new( + args.options, + #[cfg(feature = "kafka")] + args.kafka, + Arc::new(args.storage), + ) + } + StorageOptions::S3(args) => Parseable::new( + args.options, + #[cfg(feature = "kafka")] + args.kafka, + Arc::new(args.storage), + ), + StorageOptions::Blob(args) => Parseable::new( + args.options, + #[cfg(feature = "kafka")] + args.kafka, + Arc::new(args.storage), + ), +}); + +/// All state related to parseable, in one place. +pub struct Parseable { + /// Configuration variables for parseable + pub options: Arc, + /// Storage engine backing parseable + pub storage: Arc, + /// Metadata and staging realting to each logstreams + /// A globally shared mapping of `Streams` that parseable is aware of. + pub streams: Streams, + /// Metadata associated only with an ingestor + pub ingestor_metadata: Option>, + /// Used to configure the kafka connector + #[cfg(feature = "kafka")] + pub kafka_config: KafkaConfig, +} + +impl Parseable { + pub fn new( + options: Options, + #[cfg(feature = "kafka")] kafka_config: KafkaConfig, + storage: Arc, + ) -> Self { + let ingestor_metadata = match &options.mode { + Mode::Ingest => Some(IngestorMetadata::load()), + _ => None, + }; + Parseable { + options: Arc::new(options), + storage, + streams: Streams::default(), + ingestor_metadata, + #[cfg(feature = "kafka")] + kafka_config, + } + } + + /// Try to get the handle of a stream in staging, if it doesn't exist return `None`. + pub fn get_stream(&self, stream_name: &str) -> Result { + self.streams + .read() + .unwrap() + .get(stream_name) + .ok_or_else(|| StreamNotFound(stream_name.to_owned())) + .cloned() + } + + /// Get the handle to a stream in staging, create one if it doesn't exist + pub fn get_or_create_stream(&self, stream_name: &str) -> StreamRef { + if let Ok(staging) = self.get_stream(stream_name) { + return staging; + } + + // Gets write privileges only for creating the stream when it doesn't already exist. + self.streams.create( + self.options.clone(), + stream_name.to_owned(), + LogStreamMetadata::default(), + self.ingestor_metadata + .as_ref() + .map(|meta| meta.get_ingestor_id()), + ) + } + + /// Writes all streams in staging onto disk, awaiting conversion into parquet. + /// Deletes all in memory recordbatches, freeing up rows in mem-writer. + pub fn flush_all_streams(&self) { + let streams = self.streams.read().unwrap(); + + for staging in streams.values() { + staging.flush() + } + } + + // validate the storage, if the proper path for staging directory is provided + // if the proper data directory is provided, or s3 bucket is provided etc + pub async fn validate_storage(&self) -> Result, ObjectStorageError> { + let obj_store = self.storage.get_object_store(); + let rel_path = parseable_json_path(); + let mut has_parseable_json = false; + let parseable_json_result = obj_store.get_object(&rel_path).await; + if parseable_json_result.is_ok() { + has_parseable_json = true; + } + + // Lists all the directories in the root of the bucket/directory + // can be a stream (if it contains .stream.json file) or not + let has_dirs = match obj_store.list_dirs().await { + Ok(dirs) => !dirs.is_empty(), + Err(_) => false, + }; + + let has_streams = obj_store.list_streams().await.is_ok(); + if !has_dirs && !has_parseable_json { + return Ok(None); + } + if has_streams { + return Ok(Some(parseable_json_result.unwrap())); + } + + if self.storage.name() == "drive" { + return Err(ObjectStorageError::Custom(format!("Could not start the server because directory '{}' contains stale data, please use an empty directory, and restart the server.\n{}", self.storage.get_endpoint(), JOIN_COMMUNITY))); + } + + // S3 bucket mode + Err(ObjectStorageError::Custom(format!("Could not start the server because bucket '{}' contains stale data, please use an empty bucket and restart the server.\n{}", self.storage.get_endpoint(), JOIN_COMMUNITY))) + } + + pub fn storage(&self) -> Arc { + self.storage.clone() + } + + pub fn staging_dir(&self) -> &PathBuf { + &self.options.local_staging_path + } + + pub fn hot_tier_dir(&self) -> &Option { + &self.options.hot_tier_storage_path + } + + // returns the string representation of the storage mode + // drive --> Local drive + // s3 --> S3 bucket + // azure_blob --> Azure Blob Storage + pub fn get_storage_mode_string(&self) -> &str { + if self.storage.name() == "drive" { + return "Local drive"; + } else if self.storage.name() == "s3" { + return "S3 bucket"; + } else if self.storage.name() == "blob_store" { + return "Azure Blob Storage"; + } + "Unknown" + } + + pub fn get_server_mode_string(&self) -> &str { + match self.options.mode { + Mode::Query => "Distributed (Query)", + Mode::Ingest => "Distributed (Ingest)", + Mode::All => "Standalone", + } + } + + // create the ingestor metadata and put the .ingestor.json file in the object store + pub async fn store_ingestor_metadata(&self) -> anyhow::Result<()> { + let Some(meta) = self.ingestor_metadata.as_ref() else { + return Ok(()); + }; + let storage_ingestor_metadata = meta.migrate().await?; + let store = self.storage.get_object_store(); + + // use the id that was generated/found in the staging and + // generate the path for the object store + let path = meta.file_path(); + + // we are considering that we can always get from object store + if let Some(mut store_data) = storage_ingestor_metadata { + if store_data.domain_name != meta.domain_name { + store_data.domain_name.clone_from(&meta.domain_name); + store_data.port.clone_from(&meta.port); + + let resource = Bytes::from(serde_json::to_vec(&store_data)?); + + // if pushing to object store fails propagate the error + store.put_object(&path, resource).await?; + } + } else { + let resource = serde_json::to_vec(&meta)?.into(); + + store.put_object(&path, resource).await?; + } + + Ok(()) + } + + /// list all streams from storage + /// if stream exists in storage, create stream and schema from storage + /// and add it to the memory map + pub async fn create_stream_and_schema_from_storage( + &self, + stream_name: &str, + ) -> Result { + // Proceed to create log stream if it doesn't exist + let storage = self.storage.get_object_store(); + let streams = storage.list_streams().await?; + if !streams.contains(stream_name) { + return Ok(false); + } + + let mut stream_metadata = ObjectStoreFormat::default(); + let stream_metadata_bytes = storage.create_stream_from_ingestor(stream_name).await?; + if !stream_metadata_bytes.is_empty() { + stream_metadata = serde_json::from_slice::(&stream_metadata_bytes)?; + } + + let mut schema = Arc::new(Schema::empty()); + let schema_bytes = storage.create_schema_from_ingestor(stream_name).await?; + if !schema_bytes.is_empty() { + schema = serde_json::from_slice::>(&schema_bytes)?; + } + + let static_schema: HashMap> = schema + .fields + .into_iter() + .map(|field| (field.name().to_string(), field.clone())) + .collect(); + + let created_at = stream_metadata.created_at; + let time_partition = stream_metadata.time_partition.unwrap_or_default(); + let time_partition_limit = stream_metadata + .time_partition_limit + .and_then(|limit| limit.parse().ok()); + let custom_partition = stream_metadata.custom_partition; + let static_schema_flag = stream_metadata.static_schema_flag; + let stream_type = stream_metadata.stream_type; + let schema_version = stream_metadata.schema_version; + let log_source = stream_metadata.log_source; + let metadata = LogStreamMetadata::new( + created_at, + time_partition, + time_partition_limit, + custom_partition, + static_schema_flag, + static_schema, + stream_type, + schema_version, + log_source, + ); + self.streams.create( + self.options.clone(), + stream_name.to_string(), + metadata, + self.ingestor_metadata + .as_ref() + .map(|meta| meta.get_ingestor_id()), + ); + + Ok(true) + } + + pub async fn create_internal_stream_if_not_exists(&self) -> Result<(), StreamError> { + match self + .create_stream_if_not_exists( + INTERNAL_STREAM_NAME, + StreamType::Internal, + LogSource::Pmeta, + ) + .await + { + Err(_) | Ok(true) => return Ok(()), + _ => {} + } + + let mut header_map = HeaderMap::new(); + header_map.insert( + HeaderName::from_str(STREAM_TYPE_KEY).unwrap(), + HeaderValue::from_str(&StreamType::Internal.to_string()).unwrap(), + ); + header_map.insert(CONTENT_TYPE, HeaderValue::from_static("application/json")); + sync_streams_with_ingestors(header_map, Bytes::new(), INTERNAL_STREAM_NAME).await?; + + Ok(()) + } + + // Check if the stream exists and create a new stream if doesn't exist + pub async fn create_stream_if_not_exists( + &self, + stream_name: &str, + stream_type: StreamType, + log_source: LogSource, + ) -> Result { + if self.streams.contains(stream_name) { + return Ok(true); + } + + // For distributed deployments, if the stream not found in memory map, + //check if it exists in the storage + //create stream and schema from storage + if self.options.mode != Mode::All + && self + .create_stream_and_schema_from_storage(stream_name) + .await? + { + return Ok(false); + } + + self.create_stream( + stream_name.to_string(), + "", + None, + None, + false, + Arc::new(Schema::empty()), + stream_type, + log_source, + ) + .await?; + + Ok(false) + } + + pub async fn create_update_stream( + &self, + headers: &HeaderMap, + body: &Bytes, + stream_name: &str, + ) -> Result { + let PutStreamHeaders { + time_partition, + time_partition_limit, + custom_partition, + static_schema_flag, + update_stream_flag, + stream_type, + log_source, + } = headers.into(); + + let stream_in_memory_dont_update = + self.streams.contains(stream_name) && !update_stream_flag; + let stream_in_storage_only_for_query_node = !self.streams.contains(stream_name) // check if stream in storage only if not in memory + && self.options.mode == Mode::Query // and running in query mode + && self + .create_stream_and_schema_from_storage(stream_name) + .await?; + if stream_in_memory_dont_update || stream_in_storage_only_for_query_node { + return Err(StreamError::Custom { + msg: format!( + "Logstream {stream_name} already exists, please create a new log stream with unique name" + ), + status: StatusCode::BAD_REQUEST, + }); + } + + if update_stream_flag { + return self + .update_stream( + headers, + stream_name, + &time_partition, + static_schema_flag, + &time_partition_limit, + custom_partition.as_ref(), + ) + .await; + } + + let time_partition_in_days = if !time_partition_limit.is_empty() { + Some(validate_time_partition_limit(&time_partition_limit)?) + } else { + None + }; + + if let Some(custom_partition) = &custom_partition { + validate_custom_partition(custom_partition)?; + } + + if !time_partition.is_empty() && custom_partition.is_some() { + let custom_partition_list = custom_partition + .as_ref() + .unwrap() + .split(',') + .collect::>(); + if custom_partition_list.contains(&time_partition.as_str()) { + return Err(CreateStreamError::Custom { + msg: format!( + "time partition {} cannot be set as custom partition", + time_partition + ), + status: StatusCode::BAD_REQUEST, + } + .into()); + } + } + + let schema = validate_static_schema( + body, + stream_name, + &time_partition, + custom_partition.as_ref(), + static_schema_flag, + )?; + + self.create_stream( + stream_name.to_string(), + &time_partition, + time_partition_in_days, + custom_partition.as_ref(), + static_schema_flag, + schema, + stream_type, + log_source, + ) + .await?; + + Ok(headers.clone()) + } + + async fn update_stream( + &self, + headers: &HeaderMap, + stream_name: &str, + time_partition: &str, + static_schema_flag: bool, + time_partition_limit: &str, + custom_partition: Option<&String>, + ) -> Result { + if !self.streams.contains(stream_name) { + return Err(StreamNotFound(stream_name.to_string()).into()); + } + if !time_partition.is_empty() { + return Err(StreamError::Custom { + msg: "Altering the time partition of an existing stream is restricted.".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + if static_schema_flag { + return Err(StreamError::Custom { + msg: "Altering the schema of an existing stream is restricted.".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + if !time_partition_limit.is_empty() { + let time_partition_days = validate_time_partition_limit(time_partition_limit)?; + self.update_time_partition_limit_in_stream( + stream_name.to_string(), + time_partition_days, + ) + .await?; + return Ok(headers.clone()); + } + self.validate_and_update_custom_partition(stream_name, custom_partition) + .await?; + + Ok(headers.clone()) + } + + #[allow(clippy::too_many_arguments)] + pub async fn create_stream( + &self, + stream_name: String, + time_partition: &str, + time_partition_limit: Option, + custom_partition: Option<&String>, + static_schema_flag: bool, + schema: Arc, + stream_type: StreamType, + log_source: LogSource, + ) -> Result<(), CreateStreamError> { + // fail to proceed if invalid stream name + if stream_type != StreamType::Internal { + validator::stream_name(&stream_name, stream_type)?; + } + // Proceed to create log stream if it doesn't exist + let storage = self.storage.get_object_store(); + + let meta = ObjectStoreFormat { + created_at: Local::now().to_rfc3339(), + permissions: vec![Permisssion::new(PARSEABLE.options.username.clone())], + stream_type, + time_partition: (!time_partition.is_empty()).then(|| time_partition.to_string()), + time_partition_limit: time_partition_limit.map(|limit| limit.to_string()), + custom_partition: custom_partition.cloned(), + static_schema_flag, + schema_version: SchemaVersion::V1, // NOTE: Newly created streams are all V1 + owner: Owner { + id: PARSEABLE.options.username.clone(), + group: PARSEABLE.options.username.clone(), + }, + log_source: log_source.clone(), + ..Default::default() + }; + + match storage + .create_stream(&stream_name, meta, schema.clone()) + .await + { + Ok(created_at) => { + let mut static_schema: HashMap> = HashMap::new(); + + for (field_name, field) in schema + .fields() + .iter() + .map(|field| (field.name().to_string(), field.clone())) + { + static_schema.insert(field_name, field); + } + + let metadata = LogStreamMetadata::new( + created_at, + time_partition.to_owned(), + time_partition_limit, + custom_partition.cloned(), + static_schema_flag, + static_schema, + stream_type, + SchemaVersion::V1, // New stream + log_source, + ); + self.streams.create( + self.options.clone(), + stream_name.to_string(), + metadata, + self.ingestor_metadata + .as_ref() + .map(|meta| meta.get_ingestor_id()), + ); + } + Err(err) => { + return Err(CreateStreamError::Storage { stream_name, err }); + } + } + Ok(()) + } + + async fn validate_and_update_custom_partition( + &self, + stream_name: &str, + custom_partition: Option<&String>, + ) -> Result<(), StreamError> { + if let Some(custom_partition) = custom_partition { + validate_custom_partition(custom_partition)?; + } + self.update_custom_partition_in_stream(stream_name.to_string(), custom_partition) + .await?; + + Ok(()) + } + + pub async fn update_time_partition_limit_in_stream( + &self, + stream_name: String, + time_partition_limit: NonZeroU32, + ) -> Result<(), CreateStreamError> { + let storage = self.storage.get_object_store(); + if let Err(err) = storage + .update_time_partition_limit_in_stream(&stream_name, time_partition_limit) + .await + { + return Err(CreateStreamError::Storage { stream_name, err }); + } + + if let Ok(stream) = self.get_stream(&stream_name) { + stream.set_time_partition_limit(time_partition_limit) + } else { + return Err(CreateStreamError::Custom { + msg: "failed to update time partition limit in metadata".to_string(), + status: StatusCode::EXPECTATION_FAILED, + }); + } + + Ok(()) + } + + pub async fn update_custom_partition_in_stream( + &self, + stream_name: String, + custom_partition: Option<&String>, + ) -> Result<(), CreateStreamError> { + let stream = self.get_stream(&stream_name).expect(STREAM_EXISTS); + let static_schema_flag = stream.get_static_schema_flag(); + let time_partition = stream.get_time_partition(); + if static_schema_flag { + let schema = stream.get_schema(); + + if let Some(custom_partition) = custom_partition { + let custom_partition_list = custom_partition.split(',').collect::>(); + for partition in custom_partition_list.iter() { + if !schema + .fields() + .iter() + .any(|field| field.name() == partition) + { + return Err(CreateStreamError::Custom { + msg: format!("custom partition field {partition} does not exist in the schema for the stream {stream_name}"), + status: StatusCode::BAD_REQUEST, + }); + } + } + + for partition in custom_partition_list { + if time_partition + .as_ref() + .is_some_and(|time| time == partition) + { + return Err(CreateStreamError::Custom { + msg: format!( + "time partition {} cannot be set as custom partition", + partition + ), + status: StatusCode::BAD_REQUEST, + }); + } + } + } + } + let storage = self.storage.get_object_store(); + if let Err(err) = storage + .update_custom_partition_in_stream(&stream_name, custom_partition) + .await + { + return Err(CreateStreamError::Storage { stream_name, err }); + } + + stream.set_custom_partition(custom_partition); + + Ok(()) + } + + /// Updates the first-event-at in storage and logstream metadata for the specified stream. + /// + /// This function updates the `first-event-at` in both the object store and the stream info metadata. + /// If either update fails, an error is logged, but the function will still return the `first-event-at`. + /// + /// # Arguments + /// + /// * `stream_name` - The name of the stream to update. + /// * `first_event_at` - The value of first-event-at. + /// + /// # Returns + /// + /// * `Option` - Returns `Some(String)` with the provided timestamp if the update is successful, + /// or `None` if an error occurs. + /// + /// # Errors + /// + /// This function logs an error if: + /// * The `first-event-at` cannot be updated in the object store. + /// * The `first-event-at` cannot be updated in the stream info. + /// + /// # Examples + ///```ignore + /// ```rust + /// use parseable::handlers::http::modal::utils::logstream_utils::update_first_event_at; + /// let result = update_first_event_at("my_stream", "2023-01-01T00:00:00Z").await; + /// match result { + /// Some(timestamp) => println!("first-event-at: {}", timestamp), + /// None => eprintln!("Failed to update first-event-at"), + /// } + /// ``` + pub async fn update_first_event_at( + &self, + stream_name: &str, + first_event_at: &str, + ) -> Option { + let storage = self.storage.get_object_store(); + if let Err(err) = storage + .update_first_event_in_stream(stream_name, first_event_at) + .await + { + error!( + "Failed to update first_event_at in storage for stream {:?}: {err:?}", + stream_name + ); + } + + match self.get_stream(stream_name) { + Ok(stream) => stream.set_first_event_at(first_event_at), + Err(err) => error!( + "Failed to update first_event_at in stream info for stream {:?}: {err:?}", + stream_name + ), + } + + Some(first_event_at.to_string()) + } +} + +pub fn validate_static_schema( + body: &Bytes, + stream_name: &str, + time_partition: &str, + custom_partition: Option<&String>, + static_schema_flag: bool, +) -> Result, CreateStreamError> { + if !static_schema_flag { + return Ok(Arc::new(Schema::empty())); + } + + if body.is_empty() { + return Err(CreateStreamError::Custom { + msg: format!( + "Please provide schema in the request body for static schema logstream {stream_name}" + ), + status: StatusCode::BAD_REQUEST, + }); + } + + let static_schema: StaticSchema = serde_json::from_slice(body)?; + let parsed_schema = + convert_static_schema_to_arrow_schema(static_schema, time_partition, custom_partition) + .map_err(|_| CreateStreamError::Custom { + msg: format!("Unable to commit static schema, logstream {stream_name} not created"), + status: StatusCode::BAD_REQUEST, + })?; + + Ok(parsed_schema) +} + +pub fn validate_time_partition_limit( + time_partition_limit: &str, +) -> Result { + if !time_partition_limit.ends_with('d') { + return Err(CreateStreamError::Custom { + msg: "Missing 'd' suffix for duration value".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + let days = &time_partition_limit[0..time_partition_limit.len() - 1]; + let Ok(days) = days.parse::() else { + return Err(CreateStreamError::Custom { + msg: "Could not convert duration to an unsigned number".to_string(), + status: StatusCode::BAD_REQUEST, + }); + }; + + Ok(days) +} + +pub fn validate_custom_partition(custom_partition: &str) -> Result<(), CreateStreamError> { + let custom_partition_list = custom_partition.split(',').collect::>(); + if custom_partition_list.len() > 3 { + return Err(CreateStreamError::Custom { + msg: "Maximum 3 custom partition keys are supported".to_string(), + status: StatusCode::BAD_REQUEST, + }); + } + Ok(()) +} diff --git a/src/staging/mod.rs b/src/parseable/staging/mod.rs similarity index 70% rename from src/staging/mod.rs rename to src/parseable/staging/mod.rs index f7b81b9a9..256133841 100644 --- a/src/staging/mod.rs +++ b/src/parseable/staging/mod.rs @@ -17,14 +17,8 @@ * */ -use once_cell::sync::Lazy; -pub use streams::{Stream, Streams}; - -use crate::metadata::error::stream_info::MetadataError; - -mod reader; -mod streams; -mod writer; +pub mod reader; +pub mod writer; #[derive(Debug, thiserror::Error)] pub enum StagingError { @@ -36,10 +30,6 @@ pub enum StagingError { ObjectStorage(#[from] std::io::Error), #[error("Could not generate parquet file")] Create, - #[error("Metadata Error: {0}")] - Metadata(#[from] MetadataError), + // #[error("Metadata Error: {0}")] + // Metadata(#[from] MetadataError), } - -/// Staging is made up of multiple streams, each stream's context is housed in a single `Stream` object. -/// `STAGING` is a globally shared mapping of `Streams` that are in staging. -pub static STAGING: Lazy = Lazy::new(Streams::default); diff --git a/src/staging/reader.rs b/src/parseable/staging/reader.rs similarity index 100% rename from src/staging/reader.rs rename to src/parseable/staging/reader.rs diff --git a/src/staging/writer.rs b/src/parseable/staging/writer.rs similarity index 100% rename from src/staging/writer.rs rename to src/parseable/staging/writer.rs diff --git a/src/staging/streams.rs b/src/parseable/streams.rs similarity index 66% rename from src/staging/streams.rs rename to src/parseable/streams.rs index e04b3f744..7d8decdca 100644 --- a/src/staging/streams.rs +++ b/src/parseable/streams.rs @@ -19,7 +19,8 @@ use std::{ collections::HashMap, - fs::{self, remove_file, File, OpenOptions}, + fs::{remove_file, write, File, OpenOptions}, + num::NonZeroU32, path::{Path, PathBuf}, process, sync::{Arc, Mutex, RwLock}, @@ -27,17 +28,14 @@ use std::{ use arrow_array::RecordBatch; use arrow_ipc::writer::StreamWriter; -use arrow_schema::Schema; +use arrow_schema::{Field, Fields, Schema}; use chrono::{NaiveDateTime, Timelike, Utc}; use derive_more::{Deref, DerefMut}; use itertools::Itertools; use parquet::{ arrow::ArrowWriter, basic::Encoding, - file::{ - properties::{WriterProperties, WriterPropertiesBuilder}, - FOOTER_SIZE, - }, + file::{properties::WriterProperties, FOOTER_SIZE}, format::SortingColumn, schema::types::ColumnPath, }; @@ -48,42 +46,60 @@ use tracing::{error, info, trace, warn}; use crate::{ cli::Options, event::DEFAULT_TIMESTAMP_KEY, - handlers::http::modal::ingest_server::INGESTOR_META, - metadata::{LOCK_EXPECT, STREAM_INFO}, + metadata::{LogStreamMetadata, SchemaVersion}, metrics, - option::{Mode, CONFIG}, - storage::{object_storage::to_bytes, StreamType, OBJECT_STORE_DATA_GRANULARITY}, + option::Mode, + storage::{ + object_storage::to_bytes, retention::Retention, StreamType, OBJECT_STORE_DATA_GRANULARITY, + }, utils::minute_to_slot, + LOCK_EXPECT, }; use super::{ - reader::{MergedRecordReader, MergedReverseRecordReader}, - writer::Writer, - StagingError, + staging::{ + reader::{MergedRecordReader, MergedReverseRecordReader}, + writer::Writer, + StagingError, + }, + LogStream, }; +#[derive(Debug, thiserror::Error)] +#[error("Stream not found: {0}")] +pub struct StreamNotFound(pub String); + const ARROW_FILE_EXTENSION: &str = "data.arrows"; -pub type StreamRef<'a> = Arc>; +pub type StreamRef = Arc; -/// State of staging associated with a single stream of data in parseable. -pub struct Stream<'a> { +/// All state associated with a single logstream in Parseable. +pub struct Stream { pub stream_name: String, + pub metadata: RwLock, pub data_path: PathBuf, - pub options: &'a Options, + pub options: Arc, pub writer: Mutex, + pub ingestor_id: Option, } -impl<'a> Stream<'a> { - pub fn new(options: &'a Options, stream_name: impl Into) -> StreamRef<'a> { +impl Stream { + pub fn new( + options: Arc, + stream_name: impl Into, + metadata: LogStreamMetadata, + ingestor_id: Option, + ) -> StreamRef { let stream_name = stream_name.into(); let data_path = options.local_stream_data_path(&stream_name); Arc::new(Self { stream_name, + metadata: RwLock::new(metadata), data_path, options, writer: Mutex::new(Writer::default()), + ingestor_id, }) } @@ -137,8 +153,8 @@ impl<'a> Stream<'a> { custom_partition_values: &HashMap, ) -> PathBuf { let mut hostname = hostname::get().unwrap().into_string().unwrap(); - if self.options.mode == Mode::Ingest { - hostname.push_str(&INGESTOR_META.get_ingestor_id()); + if let Some(id) = &self.ingestor_id { + hostname.push_str(id); } let filename = format!( "{}{stream_hash}.date={}.hour={:02}.minute={}.{}{hostname}.{ARROW_FILE_EXTENSION}", @@ -281,14 +297,14 @@ impl<'a> Stream<'a> { } /// Converts arrow files in staging into parquet files, does so only for past minutes when run with `!shutdown_signal` - fn prepare_parquet(&self, shutdown_signal: bool) -> Result<(), StagingError> { + pub fn prepare_parquet(&self, shutdown_signal: bool) -> Result<(), StagingError> { info!( "Starting arrow_conversion job for stream- {}", self.stream_name ); - let time_partition = STREAM_INFO.get_time_partition(&self.stream_name)?; - let custom_partition = STREAM_INFO.get_custom_partition(&self.stream_name)?; + let time_partition = self.get_time_partition(); + let custom_partition = self.get_custom_partition(); // read arrow files on disk // convert them to parquet @@ -303,8 +319,8 @@ impl<'a> Stream<'a> { // check if there is already a schema file in staging pertaining to this stream // if yes, then merge them and save - if let Some(schema) = schema { - let static_schema_flag = STREAM_INFO.get_static_schema_flag(&self.stream_name)?; + if let Some(mut schema) = schema { + let static_schema_flag = self.get_static_schema_flag(); if !static_schema_flag { // schema is dynamic, read from staging and merge if present @@ -320,16 +336,13 @@ impl<'a> Stream<'a> { self.stream_name ); staging_schemas.push(schema); - let merged_schema = Schema::try_merge(staging_schemas)?; - - warn!("writing merged schema to path- {path:?}"); - // save the merged schema on staging disk - // the path should be stream/.ingestor.id.schema - fs::write(path, to_bytes(&merged_schema))?; - } else { - info!("writing single schema to path- {path:?}"); - fs::write(path, to_bytes(&schema))?; + schema = Schema::try_merge(staging_schemas)?; } + + // save the merged schema on staging disk + // the path should be stream/.ingestor.{id}.schema + info!("writing schema to path - {path:?}"); + write(path, to_bytes(&schema))?; } } @@ -344,7 +357,7 @@ impl<'a> Stream<'a> { self.writer.lock().unwrap().mem.clear(); } - fn flush(&self) { + pub fn flush(&self) { let mut disk_writers = { let mut writer = self.writer.lock().unwrap(); // Flush memory @@ -359,6 +372,53 @@ impl<'a> Stream<'a> { } } + fn parquet_writer_props( + &self, + merged_schema: &Schema, + time_partition: Option<&String>, + custom_partition: Option<&String>, + ) -> WriterProperties { + // Determine time partition field + let time_partition_field = time_partition.map_or(DEFAULT_TIMESTAMP_KEY, |tp| tp.as_str()); + + // Find time partition index + let time_partition_idx = merged_schema.index_of(time_partition_field).unwrap_or(0); + + let mut props = WriterProperties::builder() + .set_max_row_group_size(self.options.row_group_size) + .set_compression(self.options.parquet_compression.into()) + .set_column_encoding( + ColumnPath::new(vec![time_partition_field.to_string()]), + Encoding::DELTA_BINARY_PACKED, + ); + + // Create sorting columns + let mut sorting_column_vec = vec![SortingColumn { + column_idx: time_partition_idx as i32, + descending: true, + nulls_first: true, + }]; + + // Describe custom partition column encodings and sorting + if let Some(custom_partition) = custom_partition { + for partition in custom_partition.split(',') { + if let Ok(idx) = merged_schema.index_of(partition) { + let column_path = ColumnPath::new(vec![partition.to_string()]); + props = props.set_column_encoding(column_path, Encoding::DELTA_BYTE_ARRAY); + + sorting_column_vec.push(SortingColumn { + column_idx: idx as i32, + descending: true, + nulls_first: true, + }); + } + } + } + + // Set sorting columns + props.set_sorting_columns(Some(sorting_column_vec)).build() + } + /// This function reads arrow files, groups their schemas /// /// converts them into parquet files and returns a merged schema @@ -405,13 +465,7 @@ impl<'a> Stream<'a> { } let merged_schema = record_reader.merged_schema(); - let props = parquet_writer_props( - self.options, - &merged_schema, - time_partition, - custom_partition, - ) - .build(); + let props = self.parquet_writer_props(&merged_schema, time_partition, custom_partition); schemas.push(merged_schema.clone()); let schema = Arc::new(merged_schema); let mut part_path = parquet_path.to_owned(); @@ -473,105 +527,207 @@ impl<'a> Stream<'a> { Schema::try_merge(vec![schema, current_schema]).unwrap() } -} -fn parquet_writer_props( - options: &Options, - merged_schema: &Schema, - time_partition: Option<&String>, - custom_partition: Option<&String>, -) -> WriterPropertiesBuilder { - // Determine time partition field - let time_partition_field = time_partition.map_or(DEFAULT_TIMESTAMP_KEY, |tp| tp.as_str()); - - // Find time partition index - let time_partition_idx = merged_schema.index_of(time_partition_field).unwrap_or(0); - - let mut props = WriterProperties::builder() - .set_max_row_group_size(options.row_group_size) - .set_compression(options.parquet_compression.into()) - .set_column_encoding( - ColumnPath::new(vec![time_partition_field.to_string()]), - Encoding::DELTA_BINARY_PACKED, - ); + /// Stores the provided stream metadata in memory mapping + pub async fn set_metadata(&self, updated_metadata: LogStreamMetadata) { + *self.metadata.write().expect(LOCK_EXPECT) = updated_metadata; + } - // Create sorting columns - let mut sorting_column_vec = vec![SortingColumn { - column_idx: time_partition_idx as i32, - descending: true, - nulls_first: true, - }]; - - // Describe custom partition column encodings and sorting - if let Some(custom_partition) = custom_partition { - for partition in custom_partition.split(',') { - if let Ok(idx) = merged_schema.index_of(partition) { - let column_path = ColumnPath::new(vec![partition.to_string()]); - props = props.set_column_encoding(column_path, Encoding::DELTA_BYTE_ARRAY); - - sorting_column_vec.push(SortingColumn { - column_idx: idx as i32, - descending: true, - nulls_first: true, - }); - } - } + pub fn get_first_event(&self) -> Option { + self.metadata + .read() + .expect(LOCK_EXPECT) + .first_event_at + .clone() } - // Set sorting columns - props.set_sorting_columns(Some(sorting_column_vec)) -} + pub fn get_time_partition(&self) -> Option { + self.metadata + .read() + .expect(LOCK_EXPECT) + .time_partition + .clone() + } -#[derive(Deref, DerefMut, Default)] -pub struct Streams(RwLock>>); + pub fn get_time_partition_limit(&self) -> Option { + self.metadata + .read() + .expect(LOCK_EXPECT) + .time_partition_limit + } -impl Streams { - /// Try to get the handle of a stream in staging, if it doesn't exist return `None`. - pub fn get_stream(&self, stream_name: &str) -> Option> { - self.read().unwrap().get(stream_name).cloned() + pub fn get_custom_partition(&self) -> Option { + self.metadata + .read() + .expect(LOCK_EXPECT) + .custom_partition + .clone() } - /// Get the handle to a stream in staging, create one if it doesn't exist - pub fn get_or_create_stream(&self, stream_name: &str) -> StreamRef<'static> { - if let Some(staging) = self.get_stream(stream_name) { - return staging; - } + pub fn get_static_schema_flag(&self) -> bool { + self.metadata.read().expect(LOCK_EXPECT).static_schema_flag + } + + pub fn get_retention(&self) -> Option { + self.metadata.read().expect(LOCK_EXPECT).retention.clone() + } + + pub fn get_schema_version(&self) -> SchemaVersion { + self.metadata.read().expect(LOCK_EXPECT).schema_version + } + + pub fn get_schema(&self) -> Arc { + let metadata = self.metadata.read().expect(LOCK_EXPECT); + + // sort fields on read from hashmap as order of fields can differ. + // This provides a stable output order if schema is same between calls to this function + let fields: Fields = metadata + .schema + .values() + .sorted_by_key(|field| field.name()) + .cloned() + .collect(); + + Arc::new(Schema::new(fields)) + } + + pub fn get_schema_raw(&self) -> HashMap> { + self.metadata.read().expect(LOCK_EXPECT).schema.clone() + } + + pub fn set_retention(&self, retention: Retention) { + self.metadata.write().expect(LOCK_EXPECT).retention = Some(retention); + } + + pub fn set_first_event_at(&self, first_event_at: &str) { + self.metadata.write().expect(LOCK_EXPECT).first_event_at = Some(first_event_at.to_owned()); + } + + /// Removes the `first_event_at` timestamp for the specified stream from the LogStreamMetadata. + /// + /// This function is called during the retention task, when the parquet files along with the manifest files are deleted from the storage. + /// The manifest path is removed from the snapshot in the stream.json + /// and the first_event_at value in the stream.json is removed. + /// + /// # Arguments + /// + /// * `stream_name` - The name of the stream for which the `first_event_at` timestamp is to be removed. + /// + /// # Returns + /// + /// * `Result<(), StreamNotFound>` - Returns `Ok(())` if the `first_event_at` timestamp is successfully removed, + /// or a `StreamNotFound` if the stream metadata is not found. + /// + /// # Examples + /// ```ignore + /// ```rust + /// let result = metadata.remove_first_event_at("my_stream"); + /// match result { + /// Ok(()) => println!("first-event-at removed successfully"), + /// Err(e) => eprintln!("Error removing first-event-at from PARSEABLE.streams: {}", e), + /// } + /// ``` + pub fn reset_first_event_at(&self) { + self.metadata + .write() + .expect(LOCK_EXPECT) + .first_event_at + .take(); + } + + pub fn set_time_partition_limit(&self, time_partition_limit: NonZeroU32) { + self.metadata + .write() + .expect(LOCK_EXPECT) + .time_partition_limit = Some(time_partition_limit); + } + + pub fn set_custom_partition(&self, custom_partition: Option<&String>) { + self.metadata.write().expect(LOCK_EXPECT).custom_partition = custom_partition.cloned(); + } - let staging = Stream::new(&CONFIG.options, stream_name); + pub fn set_hot_tier(&self, enable: bool) { + self.metadata.write().expect(LOCK_EXPECT).hot_tier_enabled = enable; + } - // Gets write privileges only for creating the stream when it doesn't already exist. + pub fn get_stream_type(&self) -> StreamType { + self.metadata.read().expect(LOCK_EXPECT).stream_type + } +} + +#[derive(Deref, DerefMut, Default)] +pub struct Streams(RwLock>); + +// PARSEABLE.streams should be updated +// 1. During server start up +// 2. When a new stream is created (make a new entry in the map) +// 3. When a stream is deleted (remove the entry from the map) +// 4. When first event is sent to stream (update the schema) +// 5. When set alert API is called (update the alert) +impl Streams { + pub fn create( + &self, + options: Arc, + stream_name: String, + metadata: LogStreamMetadata, + ingestor_id: Option, + ) -> StreamRef { + let stream = Stream::new(options, &stream_name, metadata, ingestor_id); self.write() - .unwrap() - .insert(stream_name.to_owned(), staging.clone()); + .expect(LOCK_EXPECT) + .insert(stream_name, stream.clone()); - staging + stream } - pub fn clear(&self, stream_name: &str) { - if let Some(stream) = self.write().unwrap().get(stream_name) { - stream.clear(); - } + /// TODO: validate possibility of stream continuing to exist despite being deleted + pub fn delete(&self, stream_name: &str) { + self.write().expect(LOCK_EXPECT).remove(stream_name); } - pub fn delete_stream(&self, stream_name: &str) { - self.write().unwrap().remove(stream_name); + pub fn contains(&self, stream_name: &str) -> bool { + self.read().expect(LOCK_EXPECT).contains_key(stream_name) } - pub fn flush_all(&self) { - let streams = self.read().unwrap(); + /// Returns the number of logstreams that parseable is aware of + pub fn len(&self) -> usize { + self.read().expect(LOCK_EXPECT).len() + } - for staging in streams.values() { - staging.flush() - } + /// Returns true if parseable is not aware of any streams + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + /// Listing of logstream names that parseable is aware of + pub fn list(&self) -> Vec { + self.read() + .expect(LOCK_EXPECT) + .keys() + .map(String::clone) + .collect() + } + + pub fn list_internal_streams(&self) -> Vec { + let map = self.read().expect(LOCK_EXPECT); + + map.iter() + .filter(|(_, stream)| { + stream.metadata.read().expect(LOCK_EXPECT).stream_type == StreamType::Internal + }) + .map(|(k, _)| k.clone()) + .collect() } /// Convert arrow files into parquet, preparing it for upload pub fn prepare_parquet(&self, shutdown_signal: bool) -> Result<(), StagingError> { - if !Path::new(&CONFIG.staging_dir()).exists() { - return Ok(()); - } - - for stream in self.read().expect(LOCK_EXPECT).values() { + let streams: Vec> = self + .read() + .expect(LOCK_EXPECT) + .values() + .map(Arc::clone) + .collect(); + for stream in streams { stream .prepare_parquet(shutdown_signal) .inspect_err(|err| error!("Failed to run conversion task {err:?}"))?; @@ -597,8 +753,13 @@ mod tests { fn test_staging_new_with_valid_stream() { let stream_name = "test_stream"; - let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let options = Arc::new(Options::default()); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); assert_eq!( staging.data_path, @@ -610,8 +771,13 @@ mod tests { fn test_staging_with_special_characters() { let stream_name = "test_stream_!@#$%^&*()"; - let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let options = Arc::new(Options::default()); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); assert_eq!( staging.data_path, @@ -623,8 +789,13 @@ mod tests { fn test_staging_data_path_initialization() { let stream_name = "example_stream"; - let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let options = Arc::new(Options::default()); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); assert_eq!( staging.data_path, @@ -636,8 +807,13 @@ mod tests { fn test_staging_with_alphanumeric_stream_name() { let stream_name = "test123stream"; - let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let options = Arc::new(Options::default()); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); assert_eq!( staging.data_path, @@ -653,7 +829,12 @@ mod tests { local_staging_path: temp_dir.path().to_path_buf(), ..Default::default() }; - let staging = Stream::new(&options, "test_stream"); + let staging = Stream::new( + Arc::new(options), + "test_stream", + LogStreamMetadata::default(), + None, + ); let files = staging.arrow_files(); @@ -671,7 +852,12 @@ mod tests { let custom_partition_values = HashMap::new(); let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let staging = Stream::new( + Arc::new(options), + stream_name, + LogStreamMetadata::default(), + None, + ); let expected_path = staging.data_path.join(format!( "{}{stream_hash}.date={}.hour={:02}.minute={}.{}.{ARROW_FILE_EXTENSION}", @@ -701,7 +887,12 @@ mod tests { custom_partition_values.insert("key2".to_string(), "value2".to_string()); let options = Options::default(); - let staging = Stream::new(&options, stream_name); + let staging = Stream::new( + Arc::new(options), + stream_name, + LogStreamMetadata::default(), + None, + ); let expected_path = staging.data_path.join(format!( "{}{stream_hash}.date={}.hour={:02}.minute={}.key1=value1.key2=value2.{}.{ARROW_FILE_EXTENSION}", @@ -726,8 +917,13 @@ mod tests { ..Default::default() }; let stream = "test_stream".to_string(); - let result = - Stream::new(&options, &stream).convert_disk_files_to_parquet(None, None, false)?; + let result = Stream::new( + Arc::new(options), + &stream, + LogStreamMetadata::default(), + None, + ) + .convert_disk_files_to_parquet(None, None, false)?; assert!(result.is_none()); // Verify metrics were set to 0 let staging_files = metrics::STAGING_FILES.with_label_values(&[&stream]).get(); @@ -773,12 +969,17 @@ mod tests { fn different_minutes_multiple_arrow_files_to_parquet() { let temp_dir = TempDir::new().unwrap(); let stream_name = "test_stream"; - let options = Options { + let options = Arc::new(Options { local_staging_path: temp_dir.path().to_path_buf(), row_group_size: 1048576, ..Default::default() - }; - let staging = Stream::new(&options, stream_name); + }); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); // Create test arrow files let schema = Schema::new(vec![ @@ -799,7 +1000,7 @@ mod tests { drop(staging); // Start with a fresh staging - let staging = Stream::new(&options, stream_name); + let staging = Stream::new(options, stream_name, LogStreamMetadata::default(), None); let result = staging .convert_disk_files_to_parquet(None, None, true) .unwrap(); @@ -817,12 +1018,17 @@ mod tests { fn same_minute_multiple_arrow_files_to_parquet() { let temp_dir = TempDir::new().unwrap(); let stream_name = "test_stream"; - let options = Options { + let options = Arc::new(Options { local_staging_path: temp_dir.path().to_path_buf(), row_group_size: 1048576, ..Default::default() - }; - let staging: Arc> = Stream::new(&options, stream_name); + }); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); // Create test arrow files let schema = Schema::new(vec![ @@ -843,7 +1049,7 @@ mod tests { drop(staging); // Start with a fresh staging - let staging = Stream::new(&options, stream_name); + let staging = Stream::new(options, stream_name, LogStreamMetadata::default(), None); let result = staging .convert_disk_files_to_parquet(None, None, true) .unwrap(); @@ -861,12 +1067,17 @@ mod tests { async fn miss_current_arrow_file_when_converting_to_parquet() { let temp_dir = TempDir::new().unwrap(); let stream_name = "test_stream"; - let options = Options { + let options = Arc::new(Options { local_staging_path: temp_dir.path().to_path_buf(), row_group_size: 1048576, ..Default::default() - }; - let staging = Stream::new(&options, stream_name); + }); + let staging = Stream::new( + options.clone(), + stream_name, + LogStreamMetadata::default(), + None, + ); // Create test arrow files let schema = Schema::new(vec![ @@ -892,7 +1103,7 @@ mod tests { drop(staging); // Start with a fresh staging - let staging = Stream::new(&options, stream_name); + let staging = Stream::new(options, stream_name, LogStreamMetadata::default(), None); let result = staging .convert_disk_files_to_parquet(None, None, false) .unwrap(); diff --git a/src/query/mod.rs b/src/query/mod.rs index 5006077a0..44dcca003 100644 --- a/src/query/mod.rs +++ b/src/query/mod.rs @@ -52,12 +52,13 @@ use crate::catalog::snapshot::Snapshot; use crate::catalog::Snapshot as CatalogSnapshot; use crate::event; use crate::handlers::http::query::QueryError; -use crate::metadata::STREAM_INFO; -use crate::option::{Mode, CONFIG}; +use crate::option::Mode; +use crate::parseable::PARSEABLE; use crate::storage::{ObjectStorageProvider, ObjectStoreFormat, STREAM_ROOT_DIRECTORY}; use crate::utils::time::TimeRange; + pub static QUERY_SESSION: Lazy = - Lazy::new(|| Query::create_session_context(CONFIG.storage())); + Lazy::new(|| Query::create_session_context(PARSEABLE.storage())); // A query request by client #[derive(Debug)] @@ -74,7 +75,7 @@ impl Query { .get_datafusion_runtime() .with_disk_manager(DiskManagerConfig::NewOs); - let (pool_size, fraction) = match CONFIG.options.query_memory_pool_size { + let (pool_size, fraction) = match PARSEABLE.options.query_memory_pool_size { Some(size) => (size, 1.), None => { let mut system = System::new(); @@ -138,7 +139,7 @@ impl Query { &self, stream_name: String, ) -> Result<(Vec, Vec), ExecuteError> { - let time_partition = STREAM_INFO.get_time_partition(&stream_name)?; + let time_partition = PARSEABLE.get_stream(&stream_name)?.get_time_partition(); let df = QUERY_SESSION .execute_logical_plan(self.final_logical_plan(&time_partition)) @@ -161,7 +162,7 @@ impl Query { } pub async fn get_dataframe(&self, stream_name: String) -> Result { - let time_partition = STREAM_INFO.get_time_partition(&stream_name)?; + let time_partition = PARSEABLE.get_stream(&stream_name)?.get_time_partition(); let df = QUERY_SESSION .execute_logical_plan(self.final_logical_plan(&time_partition)) @@ -287,9 +288,10 @@ impl CountsRequest { /// get the sum of `num_rows` between the `startTime` and `endTime`, /// divide that by number of bins and return in a manner acceptable for the console pub async fn get_bin_density(&self) -> Result, QueryError> { - let time_partition = STREAM_INFO - .get_time_partition(&self.stream.clone()) + let time_partition = PARSEABLE + .get_stream(&self.stream) .map_err(|err| anyhow::Error::msg(err.to_string()))? + .get_time_partition() .unwrap_or_else(|| event::DEFAULT_TIMESTAMP_KEY.to_owned()); // get time range @@ -424,7 +426,7 @@ pub async fn get_manifest_list( stream_name: &str, time_range: &TimeRange, ) -> Result, QueryError> { - let glob_storage = CONFIG.storage().get_object_store(); + let glob_storage = PARSEABLE.storage.get_object_store(); let object_store = QUERY_SESSION .state() @@ -443,7 +445,7 @@ pub async fn get_manifest_list( let mut merged_snapshot: Snapshot = Snapshot::default(); // get a list of manifests - if CONFIG.options.mode == Mode::Query { + if PARSEABLE.options.mode == Mode::Query { let path = RelativePathBuf::from_iter([stream_name, STREAM_ROOT_DIRECTORY]); let obs = glob_storage .get_objects( @@ -614,7 +616,7 @@ pub fn flatten_objects_for_count(objects: Vec) -> Vec { } pub mod error { - use crate::{metadata::error::stream_info::MetadataError, storage::ObjectStorageError}; + use crate::{parseable::StreamNotFound, storage::ObjectStorageError}; use datafusion::error::DataFusionError; #[derive(Debug, thiserror::Error)] @@ -623,8 +625,8 @@ pub mod error { ObjectStorage(#[from] ObjectStorageError), #[error("Query Execution failed due to error in datafusion: {0}")] Datafusion(#[from] DataFusionError), - #[error("Query Execution failed due to error in fetching metadata: {0}")] - Metadata(#[from] MetadataError), + #[error("{0}")] + StreamNotFound(#[from] StreamNotFound), } } diff --git a/src/query/stream_schema_provider.rs b/src/query/stream_schema_provider.rs index ae1593f16..1bf34e701 100644 --- a/src/query/stream_schema_provider.rs +++ b/src/query/stream_schema_provider.rs @@ -19,7 +19,7 @@ use crate::catalog::manifest::File; use crate::hottier::HotTierManager; use crate::option::Mode; -use crate::staging::STAGING; +use crate::parseable::STREAM_EXISTS; use crate::{ catalog::snapshot::{self, Snapshot}, storage::{ObjectStoreFormat, STREAM_ROOT_DIRECTORY}, @@ -65,9 +65,8 @@ use crate::{ self, column::TypedStatistics, manifest::Manifest, snapshot::ManifestItem, ManifestFile, }, event::DEFAULT_TIMESTAMP_KEY, - metadata::STREAM_INFO, metrics::QUERY_CACHE_HIT, - option::CONFIG, + parseable::PARSEABLE, storage::ObjectStorage, }; @@ -87,13 +86,16 @@ impl SchemaProvider for GlobalSchemaProvider { } fn table_names(&self) -> Vec { - STREAM_INFO.list_streams() + PARSEABLE.streams.list() } async fn table(&self, name: &str) -> DataFusionResult>> { if self.table_exist(name) { Ok(Some(Arc::new(StandardTableProvider { - schema: STREAM_INFO.schema(name).unwrap(), + schema: PARSEABLE + .get_stream(name) + .expect(STREAM_EXISTS) + .get_schema(), stream: name.to_owned(), url: self.storage.store_url(), }))) @@ -103,7 +105,7 @@ impl SchemaProvider for GlobalSchemaProvider { } fn table_exist(&self, name: &str) -> bool { - STREAM_INFO.stream_exists(name) + PARSEABLE.streams.contains(name) } } @@ -193,7 +195,7 @@ impl StandardTableProvider { let hot_tier_files = hot_tier_files .into_iter() .map(|mut file| { - let path = CONFIG + let path = PARSEABLE .options .hot_tier_storage_path .as_ref() @@ -303,7 +305,7 @@ impl StandardTableProvider { // TODO: figure out an elegant solution to this #[cfg(windows)] { - if CONFIG.storage_name.eq("drive") { + if PARSEABLE.storage.name() == "drive" { file_path = object_store::path::Path::from_absolute_path(file_path) .unwrap() .to_string(); @@ -428,7 +430,7 @@ impl TableProvider for StandardTableProvider { .object_store_registry .get_store(&self.url) .unwrap(); - let glob_storage = CONFIG.storage().get_object_store(); + let glob_storage = PARSEABLE.storage.get_object_store(); let object_store_format = glob_storage .get_object_store_format(&self.stream) @@ -441,7 +443,7 @@ impl TableProvider for StandardTableProvider { } if include_now(filters, &time_partition) { - if let Some(staging) = STAGING.get_stream(&self.stream) { + if let Ok(staging) = PARSEABLE.get_stream(&self.stream) { let records = staging.recordbatches_cloned(&self.schema); let reversed_mem_table = reversed_mem_table(records, self.schema.clone())?; @@ -452,7 +454,7 @@ impl TableProvider for StandardTableProvider { } }; let mut merged_snapshot: snapshot::Snapshot = Snapshot::default(); - if CONFIG.options.mode == Mode::Query { + if PARSEABLE.options.mode == Mode::Query { let path = RelativePathBuf::from_iter([&self.stream, STREAM_ROOT_DIRECTORY]); let obs = glob_storage .get_objects( diff --git a/src/rbac/map.rs b/src/rbac/map.rs index 652bf6bc1..02adc3463 100644 --- a/src/rbac/map.rs +++ b/src/rbac/map.rs @@ -17,7 +17,7 @@ */ use crate::rbac::user::User; -use crate::{option::CONFIG, storage::StorageMetadata}; +use crate::{parseable::PARSEABLE, storage::StorageMetadata}; use std::{collections::HashMap, sync::Mutex}; use super::Response; @@ -110,8 +110,8 @@ pub fn init(metadata: &StorageMetadata) { sessions.track_new( admin_username, SessionKey::BasicAuth { - username: CONFIG.options.username.clone(), - password: CONFIG.options.password.clone(), + username: PARSEABLE.options.username.clone(), + password: PARSEABLE.options.password.clone(), }, chrono::DateTime::::MAX_UTC, admin_permissions, diff --git a/src/rbac/user.rs b/src/rbac/user.rs index ad0d302a3..45fc50d79 100644 --- a/src/rbac/user.rs +++ b/src/rbac/user.rs @@ -25,7 +25,7 @@ use argon2::{ use rand::distributions::{Alphanumeric, DistString}; -use crate::option::CONFIG; +use crate::parseable::PARSEABLE; #[derive(Debug, Clone, PartialEq, Eq, serde::Serialize, serde::Deserialize)] #[serde(untagged)] @@ -137,8 +137,8 @@ pub struct PassCode { } pub fn get_admin_user() -> User { - let username = CONFIG.options.username.clone(); - let password = CONFIG.options.password.clone(); + let username = PARSEABLE.options.username.clone(); + let password = PARSEABLE.options.password.clone(); let hashcode = gen_hash(&password); User { diff --git a/src/static_schema.rs b/src/static_schema.rs index 6717175d0..5b1a5cada 100644 --- a/src/static_schema.rs +++ b/src/static_schema.rs @@ -59,7 +59,7 @@ pub struct Metadata {} pub fn convert_static_schema_to_arrow_schema( static_schema: StaticSchema, time_partition: &str, - custom_partition: &str, + custom_partition: Option<&String>, ) -> Result, AnyError> { let mut parsed_schema = ParsedSchema { fields: Vec::new(), @@ -67,7 +67,7 @@ pub fn convert_static_schema_to_arrow_schema( }; let mut time_partition_exists = false; - if !custom_partition.is_empty() { + if let Some(custom_partition) = custom_partition { let custom_partition_list = custom_partition.split(',').collect::>(); let mut custom_partition_exists = HashMap::with_capacity(custom_partition_list.len()); diff --git a/src/storage/azure_blob.rs b/src/storage/azure_blob.rs index b70f66bf3..b34e1e6d4 100644 --- a/src/storage/azure_blob.rs +++ b/src/storage/azure_blob.rs @@ -17,7 +17,7 @@ */ use super::object_storage::parseable_json_path; use super::{ - LogStream, ObjectStorage, ObjectStorageError, ObjectStorageProvider, PARSEABLE_ROOT_DIRECTORY, + ObjectStorage, ObjectStorageError, ObjectStorageProvider, PARSEABLE_ROOT_DIRECTORY, SCHEMA_FILE_NAME, STREAM_METADATA_FILE_NAME, STREAM_ROOT_DIRECTORY, }; use async_trait::async_trait; @@ -40,6 +40,7 @@ use super::metrics_layer::MetricLayer; use crate::handlers::http::users::USERS_ROOT_DIR; use crate::metrics::storage::azureblob::REQUEST_RESPONSE_TIME; use crate::metrics::storage::StorageMetrics; +use crate::parseable::LogStream; use object_store::limit::LimitStore; use object_store::path::Path as StorePath; use std::collections::{BTreeMap, HashMap, HashSet}; @@ -150,6 +151,10 @@ impl AzureBlobConfig { } impl ObjectStorageProvider for AzureBlobConfig { + fn name(&self) -> &'static str { + "blob-store" + } + fn get_datafusion_runtime(&self) -> RuntimeEnvBuilder { let azure = self.get_default_builder().build().unwrap(); // limit objectstore to a concurrent request limit diff --git a/src/storage/localfs.rs b/src/storage/localfs.rs index f9ac41a05..0e0d092c2 100644 --- a/src/storage/localfs.rs +++ b/src/storage/localfs.rs @@ -32,14 +32,14 @@ use relative_path::{RelativePath, RelativePathBuf}; use tokio::fs::{self, DirEntry}; use tokio_stream::wrappers::ReadDirStream; -use crate::option::validation; use crate::{ handlers::http::users::USERS_ROOT_DIR, metrics::storage::{localfs::REQUEST_RESPONSE_TIME, StorageMetrics}, }; +use crate::{option::validation, parseable::LogStream}; use super::{ - LogStream, ObjectStorage, ObjectStorageError, ObjectStorageProvider, ALERTS_ROOT_DIRECTORY, + ObjectStorage, ObjectStorageError, ObjectStorageProvider, ALERTS_ROOT_DIRECTORY, PARSEABLE_ROOT_DIRECTORY, SCHEMA_FILE_NAME, STREAM_METADATA_FILE_NAME, STREAM_ROOT_DIRECTORY, }; @@ -63,6 +63,10 @@ pub struct FSConfig { } impl ObjectStorageProvider for FSConfig { + fn name(&self) -> &'static str { + "drive" + } + fn get_datafusion_runtime(&self) -> RuntimeEnvBuilder { RuntimeEnvBuilder::new() } diff --git a/src/storage/mod.rs b/src/storage/mod.rs index 75b3e736f..e02094584 100644 --- a/src/storage/mod.rs +++ b/src/storage/mod.rs @@ -19,7 +19,9 @@ use crate::{ catalog::snapshot::Snapshot, event::format::LogSource, - metadata::{error::stream_info::MetadataError, SchemaVersion}, + metadata::SchemaVersion, + option::StandaloneWithDistributed, + parseable::StreamNotFound, stats::FullStats, utils::json::{deserialize_string_as_true, serialize_bool_as_true}, }; @@ -46,10 +48,6 @@ pub use store_metadata::{ put_remote_metadata, put_staging_metadata, resolve_parseable_metadata, StorageMetadata, }; -/// Name of a Stream -/// NOTE: this used to be a struct, flattened out for simplicity -pub type LogStream = String; - // metadata file names in a Stream prefix pub const STREAM_METADATA_FILE_NAME: &str = ".stream.json"; pub const PARSEABLE_METADATA_FILE_NAME: &str = ".parseable.json"; @@ -255,6 +253,10 @@ pub enum ObjectStorageError { UnhandledError(Box), #[error("Error: {0}")] PathError(relative_path::FromPathError), - #[error("Error: {0}")] - MetadataError(#[from] MetadataError), + + #[error("{0}")] + StreamNotFound(#[from] StreamNotFound), + + #[error("{0}")] + StandaloneWithDistributed(#[from] StandaloneWithDistributed), } diff --git a/src/storage/object_storage.rs b/src/storage/object_storage.rs index d290786d8..59ab1c699 100644 --- a/src/storage/object_storage.rs +++ b/src/storage/object_storage.rs @@ -17,26 +17,22 @@ */ use super::{ - retention::Retention, LogStream, ObjectStorageError, ObjectStoreFormat, Owner, Permisssion, - StorageMetadata, StreamType, ALERTS_ROOT_DIRECTORY, MANIFEST_FILE, - PARSEABLE_METADATA_FILE_NAME, PARSEABLE_ROOT_DIRECTORY, SCHEMA_FILE_NAME, - STREAM_METADATA_FILE_NAME, STREAM_ROOT_DIRECTORY, + retention::Retention, ObjectStorageError, ObjectStoreFormat, StorageMetadata, + ALERTS_ROOT_DIRECTORY, MANIFEST_FILE, PARSEABLE_METADATA_FILE_NAME, PARSEABLE_ROOT_DIRECTORY, + SCHEMA_FILE_NAME, STREAM_METADATA_FILE_NAME, STREAM_ROOT_DIRECTORY, }; use crate::alerts::AlertConfig; use crate::correlation::{CorrelationConfig, CorrelationError}; -use crate::event::format::LogSource; -use crate::handlers::http::modal::ingest_server::INGESTOR_META; +use crate::handlers::http::modal::ingest_server::INGESTOR_EXPECT; use crate::handlers::http::users::{CORRELATION_DIR, DASHBOARDS_DIR, FILTER_DIR, USERS_ROOT_DIR}; -use crate::metadata::SchemaVersion; use crate::metrics::{EVENTS_STORAGE_SIZE_DATE, LIFETIME_EVENTS_STORAGE_SIZE}; use crate::option::Mode; -use crate::staging::STAGING; +use crate::parseable::LogStream; use crate::{ catalog::{self, manifest::Manifest, snapshot::Snapshot}, - metadata::STREAM_INFO, metrics::{storage::StorageMetrics, STORAGE_SIZE}, - option::CONFIG, + parseable::PARSEABLE, stats::FullStats, }; @@ -44,7 +40,7 @@ use actix_web_prometheus::PrometheusMetrics; use arrow_schema::Schema; use async_trait::async_trait; use bytes::Bytes; -use chrono::{DateTime, Local, Utc}; +use chrono::{DateTime, Utc}; use datafusion::{datasource::listing::ListingTableUrl, execution::runtime_env::RuntimeEnvBuilder}; use once_cell::sync::OnceCell; use relative_path::RelativePath; @@ -74,6 +70,7 @@ pub trait ObjectStorageProvider: StorageMetrics + std::fmt::Debug + Send + Sync } fn get_endpoint(&self) -> String; fn register_store_metrics(&self, handler: &PrometheusMetrics); + fn name(&self) -> &'static str; } #[async_trait] @@ -149,42 +146,20 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { Ok(()) } - #[allow(clippy::too_many_arguments)] async fn create_stream( &self, stream_name: &str, - time_partition: &str, - time_partition_limit: Option, - custom_partition: &str, - static_schema_flag: bool, + meta: ObjectStoreFormat, schema: Arc, - stream_type: StreamType, - log_source: LogSource, ) -> Result { - let format = ObjectStoreFormat { - created_at: Local::now().to_rfc3339(), - permissions: vec![Permisssion::new(CONFIG.options.username.clone())], - stream_type, - time_partition: (!time_partition.is_empty()).then(|| time_partition.to_string()), - time_partition_limit: time_partition_limit.map(|limit| limit.to_string()), - custom_partition: (!custom_partition.is_empty()).then(|| custom_partition.to_string()), - static_schema_flag, - schema_version: SchemaVersion::V1, // NOTE: Newly created streams are all V1 - owner: Owner { - id: CONFIG.options.username.clone(), - group: CONFIG.options.username.clone(), - }, - log_source, - ..Default::default() - }; - let format_json = to_bytes(&format); + let format_json = to_bytes(&meta); self.put_object(&schema_path(stream_name), to_bytes(&schema)) .await?; self.put_object(&stream_json_path(stream_name), format_json) .await?; - Ok(format.created_at) + Ok(meta.created_at) } async fn update_time_partition_limit_in_stream( @@ -204,14 +179,10 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { async fn update_custom_partition_in_stream( &self, stream_name: &str, - custom_partition: &str, + custom_partition: Option<&String>, ) -> Result<(), ObjectStorageError> { let mut format = self.get_object_store_format(stream_name).await?; - if custom_partition.is_empty() { - format.custom_partition = None; - } else { - format.custom_partition = Some(custom_partition.to_string()); - } + format.custom_partition = custom_partition.cloned(); let format_json = to_bytes(&format); self.put_object(&stream_json_path(stream_name), format_json) .await?; @@ -375,7 +346,7 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { let mut config = serde_json::from_slice::(&bytes) .expect("parseable config is valid json"); - if CONFIG.options.mode == Mode::Ingest { + if PARSEABLE.options.mode == Mode::Ingest { config.stats = FullStats::default(); config.snapshot.manifest_list = vec![]; } @@ -674,20 +645,17 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { } async fn upload_files_from_staging(&self) -> Result<(), ObjectStorageError> { - if !Path::new(&CONFIG.staging_dir()).exists() { + if !Path::new(&PARSEABLE.staging_dir()).exists() { return Ok(()); } // get all streams - for stream in STREAM_INFO.list_streams() { - info!("Starting object_store_sync for stream- {stream}"); - - let custom_partition = STREAM_INFO - .get_custom_partition(&stream) - .map_err(|err| ObjectStorageError::UnhandledError(Box::new(err)))?; + for stream_name in PARSEABLE.streams.list() { + info!("Starting object_store_sync for stream- {stream_name}"); - let stage = STAGING.get_or_create_stream(&stream); - for file in stage.parquet_files() { + let stream = PARSEABLE.get_or_create_stream(&stream_name); + let custom_partition = stream.get_custom_partition(); + for file in stream.parquet_files() { let filename = file .file_name() .expect("only parquet files are returned by iterator") @@ -698,13 +666,13 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { file_date_part = file_date_part.split('=').collect::>()[1]; let compressed_size = file.metadata().map_or(0, |meta| meta.len()); STORAGE_SIZE - .with_label_values(&["data", &stream, "parquet"]) + .with_label_values(&["data", &stream_name, "parquet"]) .add(compressed_size as i64); EVENTS_STORAGE_SIZE_DATE - .with_label_values(&["data", &stream, "parquet", file_date_part]) + .with_label_values(&["data", &stream_name, "parquet", file_date_part]) .add(compressed_size as i64); LIFETIME_EVENTS_STORAGE_SIZE - .with_label_values(&["data", &stream, "parquet"]) + .with_label_values(&["data", &stream_name, "parquet"]) .add(compressed_size as i64); let mut file_suffix = str::replacen(filename, ".", "/", 3); @@ -717,7 +685,7 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { str::replacen(filename, ".", "/", 3 + custom_partition_list.len()); } - let stream_relative_path = format!("{stream}/{file_suffix}"); + let stream_relative_path = format!("{stream_name}/{file_suffix}"); // Try uploading the file, handle potential errors without breaking the loop if let Err(e) = self.upload_file(&stream_relative_path, &file).await { @@ -728,18 +696,18 @@ pub trait ObjectStorage: Debug + Send + Sync + 'static { let absolute_path = self .absolute_url(RelativePath::from_path(&stream_relative_path).unwrap()) .to_string(); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage().get_object_store(); let manifest = catalog::create_from_parquet_file(absolute_path.clone(), &file).unwrap(); - catalog::update_snapshot(store, &stream, manifest).await?; + catalog::update_snapshot(store, &stream_name, manifest).await?; let _ = fs::remove_file(file); } - for path in stage.schema_files() { + for path in stream.schema_files() { let file = File::open(&path)?; let schema: Schema = serde_json::from_reader(file)?; - commit_schema_to_storage(&stream, schema).await?; + commit_schema_to_storage(&stream_name, schema).await?; let _ = fs::remove_file(path); } } @@ -752,7 +720,7 @@ pub async fn commit_schema_to_storage( stream_name: &str, schema: Schema, ) -> Result<(), ObjectStorageError> { - let storage = CONFIG.storage().get_object_store(); + let storage = PARSEABLE.storage().get_object_store(); let stream_schema = storage.get_schema(stream_name).await?; let new_schema = Schema::try_merge(vec![schema, stream_schema]).unwrap(); storage.put_schema(stream_name, &new_schema).await @@ -766,13 +734,14 @@ pub fn to_bytes(any: &(impl ?Sized + serde::Serialize)) -> Bytes { } pub fn schema_path(stream_name: &str) -> RelativePathBuf { - match CONFIG.options.mode { + match &PARSEABLE.options.mode { Mode::Ingest => { - let file_name = format!( - ".ingestor.{}{}", - INGESTOR_META.ingestor_id.clone(), - SCHEMA_FILE_NAME - ); + let id = PARSEABLE + .ingestor_metadata + .as_ref() + .expect(INGESTOR_EXPECT) + .get_ingestor_id(); + let file_name = format!(".ingestor.{id}{SCHEMA_FILE_NAME}"); RelativePathBuf::from_iter([stream_name, STREAM_ROOT_DIRECTORY, &file_name]) } @@ -784,13 +753,14 @@ pub fn schema_path(stream_name: &str) -> RelativePathBuf { #[inline(always)] pub fn stream_json_path(stream_name: &str) -> RelativePathBuf { - match &CONFIG.options.mode { + match &PARSEABLE.options.mode { Mode::Ingest => { - let file_name = format!( - ".ingestor.{}{}", - INGESTOR_META.get_ingestor_id(), - STREAM_METADATA_FILE_NAME - ); + let id = PARSEABLE + .ingestor_metadata + .as_ref() + .expect(INGESTOR_EXPECT) + .get_ingestor_id(); + let file_name = format!(".ingestor.{id}{STREAM_METADATA_FILE_NAME}",); RelativePathBuf::from_iter([stream_name, STREAM_ROOT_DIRECTORY, &file_name]) } Mode::Query | Mode::All => RelativePathBuf::from_iter([ @@ -833,29 +803,16 @@ pub fn alert_json_path(alert_id: Ulid) -> RelativePathBuf { #[inline(always)] pub fn manifest_path(prefix: &str) -> RelativePathBuf { - if CONFIG.options.mode == Mode::Ingest { - let manifest_file_name = format!( - "ingestor.{}.{}", - INGESTOR_META.get_ingestor_id(), - MANIFEST_FILE - ); - RelativePathBuf::from_iter([prefix, &manifest_file_name]) - } else { - RelativePathBuf::from_iter([prefix, MANIFEST_FILE]) - } -} - -#[inline(always)] -pub fn ingestor_metadata_path(id: Option<&str>) -> RelativePathBuf { - if let Some(id) = id { - return RelativePathBuf::from_iter([ - PARSEABLE_ROOT_DIRECTORY, - &format!("ingestor.{}.json", id), - ]); + match &PARSEABLE.options.mode { + Mode::Ingest => { + let id = PARSEABLE + .ingestor_metadata + .as_ref() + .expect(INGESTOR_EXPECT) + .get_ingestor_id(); + let manifest_file_name = format!("ingestor.{id}.{MANIFEST_FILE}"); + RelativePathBuf::from_iter([prefix, &manifest_file_name]) + } + _ => RelativePathBuf::from_iter([prefix, MANIFEST_FILE]), } - - RelativePathBuf::from_iter([ - PARSEABLE_ROOT_DIRECTORY, - &format!("ingestor.{}.json", INGESTOR_META.get_ingestor_id()), - ]) } diff --git a/src/storage/retention.rs b/src/storage/retention.rs index 2cbee48c7..ce67808be 100644 --- a/src/storage/retention.rs +++ b/src/storage/retention.rs @@ -29,7 +29,7 @@ use once_cell::sync::Lazy; use tokio::task::JoinHandle; use tracing::{info, warn}; -use crate::metadata::STREAM_INFO; +use crate::parseable::PARSEABLE; type SchedulerHandle = JoinHandle<()>; @@ -45,18 +45,16 @@ pub fn init_scheduler() { let mut scheduler = AsyncScheduler::new(); let func = move || async { //get retention every day at 12 am - for stream in STREAM_INFO.list_streams() { - let retention = STREAM_INFO.get_retention(&stream); - - match retention { - Ok(config) => { - if let Some(config) = config { + for stream_name in PARSEABLE.streams.list() { + match PARSEABLE.get_stream(&stream_name) { + Ok(stream) => { + if let Some(config) = stream.get_retention() { for Task { action, days, .. } in config.tasks.into_iter() { match action { Action::Delete => { - let stream = stream.to_string(); + let stream_name = stream_name.clone(); tokio::spawn(async move { - action::delete(stream.clone(), u32::from(days)).await; + action::delete(stream_name, u32::from(days)).await; }); } }; @@ -64,7 +62,7 @@ pub fn init_scheduler() { } } Err(err) => { - warn!("failed to load retention config for {stream} due to {err:?}") + warn!("failed to load retention config for {stream_name} due to {err:?}") } }; } @@ -172,7 +170,7 @@ impl From for Vec { mod action { use crate::catalog::remove_manifest_from_snapshot; - use crate::{metadata, option::CONFIG}; + use crate::parseable::PARSEABLE; use chrono::{Days, NaiveDate, Utc}; use futures::{stream::FuturesUnordered, StreamExt}; use itertools::Itertools; @@ -181,7 +179,7 @@ mod action { pub(super) async fn delete(stream_name: String, days: u32) { info!("running retention task - delete for stream={stream_name}"); - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let retain_until = get_retain_until(Utc::now().date_naive(), days as u64); @@ -202,8 +200,8 @@ mod action { for date in dates_to_delete { let path = RelativePathBuf::from_iter([&stream_name, &date]); delete_tasks.push(async move { - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() .delete_prefix(&path) .await @@ -219,13 +217,12 @@ mod action { } } if let Ok(Some(first_event_at)) = res_remove_manifest { - if let Err(err) = - metadata::STREAM_INFO.set_first_event_at(&stream_name, &first_event_at) - { - error!( + match PARSEABLE.get_stream(&stream_name) { + Ok(stream) => stream.set_first_event_at(&first_event_at), + Err(err) => error!( "Failed to update first_event_at in streaminfo for stream {:?} {err:?}", stream_name - ); + ), } } } diff --git a/src/storage/s3.rs b/src/storage/s3.rs index 96bfb0d30..5fba3ba1b 100644 --- a/src/storage/s3.rs +++ b/src/storage/s3.rs @@ -43,11 +43,11 @@ use std::time::{Duration, Instant}; use super::metrics_layer::MetricLayer; use super::object_storage::parseable_json_path; use super::{ - LogStream, ObjectStorageProvider, SCHEMA_FILE_NAME, STREAM_METADATA_FILE_NAME, - STREAM_ROOT_DIRECTORY, + ObjectStorageProvider, SCHEMA_FILE_NAME, STREAM_METADATA_FILE_NAME, STREAM_ROOT_DIRECTORY, }; use crate::handlers::http::users::USERS_ROOT_DIR; use crate::metrics::storage::{s3::REQUEST_RESPONSE_TIME, StorageMetrics}; +use crate::parseable::LogStream; use crate::storage::{ObjectStorage, ObjectStorageError, PARSEABLE_ROOT_DIRECTORY}; use std::collections::HashMap; @@ -285,6 +285,10 @@ impl S3Config { } impl ObjectStorageProvider for S3Config { + fn name(&self) -> &'static str { + "s3" + } + fn get_datafusion_runtime(&self) -> RuntimeEnvBuilder { let s3 = self.get_default_builder().build().unwrap(); diff --git a/src/storage/store_metadata.rs b/src/storage/store_metadata.rs index fef82e094..6552333da 100644 --- a/src/storage/store_metadata.rs +++ b/src/storage/store_metadata.rs @@ -28,8 +28,8 @@ use relative_path::RelativePathBuf; use std::io; use crate::{ - metadata::error::stream_info::MetadataError, - option::{Mode, CONFIG, JOIN_COMMUNITY}, + option::Mode, + parseable::{JOIN_COMMUNITY, PARSEABLE}, rbac::{role::model::DefaultPrivilege, user::User}, storage::ObjectStorageError, utils::uid, @@ -69,11 +69,11 @@ impl Default for StorageMetadata { fn default() -> Self { Self { version: CURRENT_STORAGE_METADATA_VERSION.to_string(), - mode: CONFIG.storage_name.to_owned(), - staging: CONFIG.staging_dir().to_path_buf(), - storage: CONFIG.storage().get_endpoint(), + mode: PARSEABLE.storage.name().to_owned(), + staging: PARSEABLE.staging_dir().to_path_buf(), + storage: PARSEABLE.storage.get_endpoint(), deployment_id: uid::gen(), - server_mode: CONFIG.options.mode, + server_mode: PARSEABLE.options.mode, users: Vec::new(), streams: Vec::new(), roles: HashMap::default(), @@ -120,8 +120,8 @@ pub async fn resolve_parseable_metadata( EnvChange::None(metadata) => { // overwrite staging anyways so that it matches remote in case of any divergence overwrite_staging = true; - if CONFIG.options.mode == Mode::All { - standalone_after_distributed(metadata.server_mode)?; + if PARSEABLE.options.mode == Mode::All { + metadata.server_mode.standalone_after_distributed()?; } Ok(metadata) }, @@ -131,18 +131,18 @@ pub async fn resolve_parseable_metadata( EnvChange::NewStaging(mut metadata) => { // if server is started in ingest mode,we need to make sure that query mode has been started // i.e the metadata is updated to reflect the server mode = Query - if metadata.server_mode== Mode::All && CONFIG.options.mode == Mode::Ingest { + if metadata.server_mode== Mode::All && PARSEABLE.options.mode == Mode::Ingest { Err("Starting Ingest Mode is not allowed, Since Query Server has not been started yet") } else { - create_dir_all(CONFIG.staging_dir())?; - metadata.staging = CONFIG.staging_dir().canonicalize()?; + create_dir_all(PARSEABLE.staging_dir())?; + metadata.staging = PARSEABLE.staging_dir().canonicalize()?; // this flag is set to true so that metadata is copied to staging overwrite_staging = true; // overwrite remote in all and query mode // because staging dir has changed. - match CONFIG.options.mode { + match PARSEABLE.options.mode { Mode::All => { - standalone_after_distributed(metadata.server_mode) + metadata.server_mode.standalone_after_distributed() .map_err(|err| { ObjectStorageError::Custom(err.to_string()) })?; @@ -150,25 +150,25 @@ pub async fn resolve_parseable_metadata( }, Mode::Query => { overwrite_remote = true; - metadata.server_mode = CONFIG.options.mode; - metadata.staging = CONFIG.staging_dir().to_path_buf(); + metadata.server_mode = PARSEABLE.options.mode; + metadata.staging = PARSEABLE.staging_dir().to_path_buf(); }, Mode::Ingest => { // if ingest server is started fetch the metadata from remote // update the server mode for local metadata - metadata.server_mode = CONFIG.options.mode; - metadata.staging = CONFIG.staging_dir().to_path_buf(); + metadata.server_mode = PARSEABLE.options.mode; + metadata.staging = PARSEABLE.staging_dir().to_path_buf(); }, } Ok(metadata) } } EnvChange::CreateBoth => { - create_dir_all(CONFIG.staging_dir())?; + create_dir_all(PARSEABLE.staging_dir())?; let metadata = StorageMetadata::default(); // new metadata needs to be set // if mode is query or all then both staging and remote - match CONFIG.options.mode { + match PARSEABLE.options.mode { Mode::All | Mode::Query => overwrite_remote = true, _ => (), } @@ -184,7 +184,7 @@ pub async fn resolve_parseable_metadata( ObjectStorageError::UnhandledError(err) })?; - metadata.server_mode = CONFIG.options.mode; + metadata.server_mode = PARSEABLE.options.mode; if overwrite_remote { put_remote_metadata(&metadata).await?; } @@ -205,7 +205,7 @@ fn determine_environment( // if both staging and remote have same deployment id but different server modes if staging.deployment_id == remote.deployment_id && remote.server_mode == Mode::All - && (CONFIG.options.mode == Mode::Query || CONFIG.options.mode == Mode::Ingest) + && matches!(PARSEABLE.options.mode, Mode::Query | Mode::Ingest) { EnvChange::NewStaging(remote) } else if staging.deployment_id != remote.deployment_id { @@ -236,18 +236,8 @@ pub enum EnvChange { CreateBoth, } -fn standalone_after_distributed(remote_server_mode: Mode) -> Result<(), MetadataError> { - // standalone -> query | ingest allowed - // but query | ingest -> standalone not allowed - if remote_server_mode == Mode::Query { - return Err(MetadataError::StandaloneWithDistributed("Starting Standalone Mode is not permitted when Distributed Mode is enabled. Please restart the server with Distributed Mode enabled.".to_string())); - } - - Ok(()) -} - pub fn get_staging_metadata() -> io::Result> { - let path = RelativePathBuf::from(PARSEABLE_METADATA_FILE_NAME).to_path(CONFIG.staging_dir()); + let path = RelativePathBuf::from(PARSEABLE_METADATA_FILE_NAME).to_path(PARSEABLE.staging_dir()); let bytes = match fs::read(path) { Ok(bytes) => bytes, Err(err) => match err.kind() { @@ -262,15 +252,15 @@ pub fn get_staging_metadata() -> io::Result> { } pub async fn put_remote_metadata(metadata: &StorageMetadata) -> Result<(), ObjectStorageError> { - let client = CONFIG.storage().get_object_store(); + let client = PARSEABLE.storage.get_object_store(); client.put_metadata(metadata).await } pub fn put_staging_metadata(meta: &StorageMetadata) -> io::Result<()> { let mut staging_metadata = meta.clone(); - staging_metadata.server_mode = CONFIG.options.mode; - staging_metadata.staging = CONFIG.staging_dir().to_path_buf(); - let path = CONFIG.staging_dir().join(PARSEABLE_METADATA_FILE_NAME); + staging_metadata.server_mode = PARSEABLE.options.mode; + staging_metadata.staging = PARSEABLE.staging_dir().to_path_buf(); + let path = PARSEABLE.staging_dir().join(PARSEABLE_METADATA_FILE_NAME); let mut file = OpenOptions::new() .create(true) .truncate(true) diff --git a/src/sync.rs b/src/sync.rs index 4046485db..a05b4ca72 100644 --- a/src/sync.rs +++ b/src/sync.rs @@ -25,8 +25,7 @@ use tokio::{select, task}; use tracing::{error, info, trace, warn}; use crate::alerts::{alerts_utils, AlertConfig, AlertError}; -use crate::option::CONFIG; -use crate::staging::STAGING; +use crate::parseable::PARSEABLE; use crate::storage::LOCAL_SYNC_INTERVAL; use crate::{STORAGE_CONVERSION_INTERVAL, STORAGE_UPLOAD_INTERVAL}; @@ -146,11 +145,10 @@ pub fn object_store_sync() -> ( "object_store_sync", Duration::from_secs(15), || async { - CONFIG - .storage() + PARSEABLE + .storage .get_object_store() - .upload_files_from_staging() - .await + .upload_files_from_staging().await }, ) .await @@ -209,7 +207,7 @@ pub fn arrow_conversion() -> ( if let Err(e) = monitor_task_duration( "arrow_conversion", Duration::from_secs(30), - || async { STAGING.prepare_parquet(false) }, + || async { PARSEABLE.streams.prepare_parquet(false) }, ).await { warn!("failed to convert local arrow data to parquet. {e:?}"); @@ -262,7 +260,7 @@ pub fn run_local_sync() -> ( select! { _ = sync_interval.tick() => { trace!("Flushing Arrows to disk..."); - STAGING.flush_all(); + PARSEABLE.flush_all_streams(); }, res = &mut inbox_rx => {match res{ Ok(_) => break, diff --git a/src/users/dashboards.rs b/src/users/dashboards.rs index 46b3bbcf2..797314129 100644 --- a/src/users/dashboards.rs +++ b/src/users/dashboards.rs @@ -23,8 +23,8 @@ use serde::{Deserialize, Serialize}; use serde_json::Value; use crate::{ - metadata::LOCK_EXPECT, migration::to_bytes, option::CONFIG, - storage::object_storage::dashboard_path, utils::get_hash, + migration::to_bytes, parseable::PARSEABLE, storage::object_storage::dashboard_path, + utils::get_hash, LOCK_EXPECT, }; use super::TimeFilter; @@ -113,7 +113,7 @@ pub struct Dashboards(RwLock>); impl Dashboards { pub async fn load(&self) -> anyhow::Result<()> { let mut this = vec![]; - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let all_dashboards = store.get_all_dashboards().await.unwrap_or_default(); for (dashboard_relative_path, dashboards) in all_dashboards { for dashboard in dashboards { diff --git a/src/users/filters.rs b/src/users/filters.rs index 9b970b897..650be18a4 100644 --- a/src/users/filters.rs +++ b/src/users/filters.rs @@ -23,8 +23,8 @@ use std::sync::RwLock; use super::TimeFilter; use crate::{ - metadata::LOCK_EXPECT, migration::to_bytes, option::CONFIG, - storage::object_storage::filter_path, utils::get_hash, + migration::to_bytes, parseable::PARSEABLE, storage::object_storage::filter_path, + utils::get_hash, LOCK_EXPECT, }; pub static FILTERS: Lazy = Lazy::new(Filters::default); @@ -75,7 +75,7 @@ pub struct Filters(RwLock>); impl Filters { pub async fn load(&self) -> anyhow::Result<()> { let mut this = vec![]; - let store = CONFIG.storage().get_object_store(); + let store = PARSEABLE.storage.get_object_store(); let all_filters = store.get_all_saved_filters().await.unwrap_or_default(); for (filter_relative_path, filters) in all_filters { for filter in filters { diff --git a/src/utils/arrow/flight.rs b/src/utils/arrow/flight.rs index 8a78eb957..46336beb2 100644 --- a/src/utils/arrow/flight.rs +++ b/src/utils/arrow/flight.rs @@ -19,12 +19,9 @@ use crate::event::Event; use crate::handlers::http::ingest::push_logs_unchecked; use crate::handlers::http::query::Query as QueryJson; -use crate::metadata::STREAM_INFO; +use crate::parseable::PARSEABLE; use crate::query::stream_schema_provider::include_now; -use crate::{ - handlers::http::modal::IngestorMetadata, - option::{Mode, CONFIG}, -}; +use crate::{handlers::http::modal::IngestorMetadata, option::Mode}; use arrow_array::RecordBatch; use arrow_flight::encode::FlightDataEncoderBuilder; @@ -98,9 +95,10 @@ pub async fn append_temporary_events( Event, Status, > { - let schema = STREAM_INFO - .schema(stream_name) - .map_err(|err| Status::failed_precondition(format!("Metadata Error: {}", err)))?; + let schema = PARSEABLE + .get_stream(stream_name) + .map_err(|err| Status::failed_precondition(format!("Metadata Error: {}", err)))? + .get_schema(); let rb = concat_batches(&schema, minute_result) .map_err(|err| Status::failed_precondition(format!("ArrowError: {}", err)))?; @@ -135,7 +133,7 @@ pub fn send_to_ingester(start: i64, end: i64) -> bool { ); let ex = [Expr::BinaryExpr(ex1), Expr::BinaryExpr(ex2)]; - CONFIG.options.mode == Mode::Query && include_now(&ex, &None) + PARSEABLE.options.mode == Mode::Query && include_now(&ex, &None) } fn lit_timestamp_milli(time: i64) -> Expr { diff --git a/src/utils/mod.rs b/src/utils/mod.rs index 11943fe37..bf5b1258f 100644 --- a/src/utils/mod.rs +++ b/src/utils/mod.rs @@ -26,7 +26,7 @@ pub mod uid; pub mod update; use crate::handlers::http::rbac::RBACError; -use crate::option::CONFIG; +use crate::parseable::PARSEABLE; use crate::rbac::role::{Action, Permission}; use crate::rbac::Users; use actix::extract_session_key_from_req; @@ -56,19 +56,19 @@ pub fn minute_to_slot(minute: u32, data_granularity: u32) -> Option { } pub fn get_url() -> Url { - if CONFIG.options.ingestor_endpoint.is_empty() { + if PARSEABLE.options.ingestor_endpoint.is_empty() { return format!( "{}://{}", - CONFIG.options.get_scheme(), - CONFIG.options.address + PARSEABLE.options.get_scheme(), + PARSEABLE.options.address ) .parse::() // if the value was improperly set, this will panic before hand .unwrap_or_else(|err| { - panic!("{err}, failed to parse `{}` as Url. Please set the environment variable `P_ADDR` to `:` without the scheme (e.g., 192.168.1.1:8000). Please refer to the documentation: https://logg.ing/env for more details.", CONFIG.options.address) + panic!("{err}, failed to parse `{}` as Url. Please set the environment variable `P_ADDR` to `:` without the scheme (e.g., 192.168.1.1:8000). Please refer to the documentation: https://logg.ing/env for more details.", PARSEABLE.options.address) }); } - let ingestor_endpoint = &CONFIG.options.ingestor_endpoint; + let ingestor_endpoint = &PARSEABLE.options.ingestor_endpoint; if ingestor_endpoint.starts_with("http") { panic!("Invalid value `{}`, please set the environement variable `P_INGESTOR_ENDPOINT` to `:` without the scheme (e.g., 192.168.1.1:8000 or example.com:8000). Please refer to the documentation: https://logg.ing/env for more details.", ingestor_endpoint); @@ -95,7 +95,7 @@ pub fn get_url() -> Url { if hostname.starts_with("http") { panic!("Invalid value `{}`, please set the environement variable `{}` to `` without the scheme (e.g., 192.168.1.1 or example.com). Please refer to the documentation: https://logg.ing/env for more details.", hostname, var_hostname); } else { - hostname = format!("{}://{}", CONFIG.options.get_scheme(), hostname); + hostname = format!("{}://{}", PARSEABLE.options.get_scheme(), hostname); } } @@ -111,7 +111,7 @@ pub fn get_url() -> Url { } } - format!("{}://{}:{}", CONFIG.options.get_scheme(), hostname, port) + format!("{}://{}:{}", PARSEABLE.options.get_scheme(), hostname, port) .parse::() .expect("Valid URL") } @@ -122,13 +122,10 @@ fn get_from_env(var_to_fetch: &str) -> String { } pub fn get_ingestor_id() -> String { - let now = Utc::now().to_rfc3339().to_string(); - let mut hasher = Sha256::new(); - hasher.update(now); - let result = format!("{:x}", hasher.finalize()); - let result = result.split_at(15).0.to_string(); - debug!("Ingestor ID: {}", &result); - result + let now = Utc::now().to_rfc3339(); + let id = get_hash(&now).to_string().split_at(15).0.to_string(); + debug!("Ingestor ID: {id}"); + id } pub fn extract_datetime(path: &str) -> Option {