diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index e523fdfa371..5908b50d24d 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7317,7 +7317,6 @@ dependencies = [ "once_cell", "oneshot", "openssl", - "parquet", "percent-encoding", "proptest", "prost 0.14.3", @@ -7633,7 +7632,6 @@ dependencies = [ "chrono", "parquet", "proptest", - "prost 0.14.3", "quickwit-common", "quickwit-proto", "sea-query", diff --git a/quickwit/quickwit-cli/Cargo.toml b/quickwit/quickwit-cli/Cargo.toml index b40bf5585d3..5a468788a80 100644 --- a/quickwit/quickwit-cli/Cargo.toml +++ b/quickwit/quickwit-cli/Cargo.toml @@ -100,6 +100,7 @@ release-feature-set = [ "pprof", "quickwit-indexing/kafka", "quickwit-indexing/kinesis", + "quickwit-indexing/metrics", "quickwit-indexing/pulsar", "quickwit-indexing/sqs", "quickwit-indexing/vrl", @@ -113,6 +114,7 @@ release-feature-vendored-set = [ "openssl-support", "pprof", "quickwit-indexing/kinesis", + "quickwit-indexing/metrics", "quickwit-indexing/pulsar", "quickwit-indexing/sqs", "quickwit-indexing/vrl", @@ -126,6 +128,7 @@ release-macos-feature-vendored-set = [ "jemalloc", "openssl-support", "quickwit-indexing/kinesis", + "quickwit-indexing/metrics", "quickwit-indexing/pulsar", "quickwit-indexing/sqs", "quickwit-indexing/vrl", diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 0f34a3017e6..ff31adb6d90 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -37,7 +37,6 @@ use quickwit_config::{ TransformConfig, VecSourceParams, }; use quickwit_index_management::{IndexService, clear_cache_directory}; -use quickwit_indexing::IndexingPipeline; use quickwit_indexing::actors::{IndexingService, MergePipeline, MergeSchedulerService}; use quickwit_indexing::models::{ DetachIndexingPipeline, DetachMergePipeline, IndexingStatistics, SpawnPipeline, @@ -751,7 +750,7 @@ async fn extract_split_cli(args: ExtractSplitArgs) -> anyhow::Result<()> { /// Starts a tokio task that displays the indexing statistics /// every once in awhile. pub async fn start_statistics_reporting_loop( - pipeline_handle: ActorHandle, + pipeline_handle: quickwit_indexing::BoxPipelineHandle, is_stdin: bool, ) -> anyhow::Result { let mut stdout_handle = stdout(); diff --git a/quickwit/quickwit-common/src/lib.rs b/quickwit/quickwit-common/src/lib.rs index 65ce5df8a0b..80b178402f2 100644 --- a/quickwit/quickwit-common/src/lib.rs +++ b/quickwit/quickwit-common/src/lib.rs @@ -49,6 +49,9 @@ pub mod tower; pub mod type_map; pub mod uri; +mod metrics_specific; +pub use metrics_specific::*; + mod socket_addr_legacy_hash; use std::env; @@ -215,14 +218,6 @@ macro_rules! assert_eventually { }; } -/// Returns whether the given index ID corresponds to a metrics index. -/// -/// Metrics indexes use the Parquet/DataFusion pipeline instead of the Tantivy pipeline. -/// An index is considered a metrics index if it starts with "otel-metrics" or "metrics-". -pub fn is_metrics_index(index_id: &str) -> bool { - index_id.starts_with("otel-metrics") || index_id.starts_with("metrics-") -} - #[macro_export] macro_rules! ignore_error_kind { ($kind:path, $expr:expr) => { @@ -423,27 +418,6 @@ mod tests { assert_eq!(div_ceil_u32(0, 3), 0); } - #[test] - fn test_is_metrics_index() { - // OpenTelemetry metrics indexes - assert!(is_metrics_index("otel-metrics-v0_7")); - assert!(is_metrics_index("otel-metrics")); - assert!(is_metrics_index("otel-metrics-custom")); - - // Generic metrics indexes - assert!(is_metrics_index("metrics-default")); - assert!(is_metrics_index("metrics-")); - assert!(is_metrics_index("metrics-my-app")); - - // Non-metrics indexes - assert!(!is_metrics_index("otel-logs-v0_7")); - assert!(!is_metrics_index("otel-traces-v0_7")); - assert!(!is_metrics_index("my-index")); - assert!(!is_metrics_index("logs-default")); - assert!(!is_metrics_index("metrics")); // No hyphen after "metrics" - assert!(!is_metrics_index("my-metrics-index")); // Not prefixed - } - #[test] fn test_parse_bool_lenient() { assert_eq!(parse_bool_lenient("true"), Some(true)); diff --git a/quickwit/quickwit-common/src/metrics_specific.rs b/quickwit/quickwit-common/src/metrics_specific.rs new file mode 100644 index 00000000000..7d1e1bb4760 --- /dev/null +++ b/quickwit/quickwit-common/src/metrics_specific.rs @@ -0,0 +1,31 @@ +/// Returns whether the given index ID corresponds to a metrics index. +/// +/// Metrics indexes use the Parquet/DataFusion pipeline instead of the Tantivy pipeline. +/// An index is considered a metrics index if it starts with "otel-metrics" or "metrics-". +pub fn is_metrics_index(index_id: &str) -> bool { + index_id.starts_with("otel-metrics") || index_id.starts_with("metrics-") +} + +#[cfg(test)] +mod tests { + #[test] + fn test_is_metrics_index() { + // OpenTelemetry metrics indexes + assert!(is_metrics_index("otel-metrics-v0_7")); + assert!(is_metrics_index("otel-metrics")); + assert!(is_metrics_index("otel-metrics-custom")); + + // Generic metrics indexes + assert!(is_metrics_index("metrics-default")); + assert!(is_metrics_index("metrics-")); + assert!(is_metrics_index("metrics-my-app")); + + // Non-metrics indexes + assert!(!is_metrics_index("otel-logs-v0_7")); + assert!(!is_metrics_index("otel-traces-v0_7")); + assert!(!is_metrics_index("my-index")); + assert!(!is_metrics_index("logs-default")); + assert!(!is_metrics_index("metrics")); // No hyphen after "metrics" + assert!(!is_metrics_index("my-metrics-index")); // Not prefixed + } +} diff --git a/quickwit/quickwit-indexing/Cargo.toml b/quickwit/quickwit-indexing/Cargo.toml index 5d3a6504d29..ec6bc6703f6 100644 --- a/quickwit/quickwit-indexing/Cargo.toml +++ b/quickwit/quickwit-indexing/Cargo.toml @@ -50,7 +50,7 @@ utoipa = { workspace = true } vrl = { workspace = true, optional = true } warp = { workspace = true, optional = true } -arrow = { workspace = true } +arrow = { workspace = true, optional = true } quickwit-actors = { workspace = true } quickwit-aws = { workspace = true } @@ -62,7 +62,7 @@ quickwit-doc-mapper = { workspace = true } quickwit-ingest = { workspace = true } quickwit-metastore = { workspace = true } quickwit-opentelemetry = { workspace = true } -quickwit-parquet-engine = { workspace = true } +quickwit-parquet-engine = { workspace = true, optional = true } quickwit-proto = { workspace = true } quickwit-storage = { workspace = true } @@ -106,6 +106,7 @@ testsuite = [ "quickwit-proto/testsuite", "quickwit-storage/testsuite" ] +metrics = ["dep:arrow", "dep:quickwit-parquet-engine"] vrl = ["dep:vrl", "quickwit-config/vrl"] postgres = ["quickwit-metastore/postgres"] ci-test = [] @@ -114,7 +115,6 @@ ci-test = [] bytes = { workspace = true } criterion = { workspace = true, features = ["async_tokio"] } mockall = { workspace = true } -parquet = { workspace = true } proptest = { workspace = true } prost = { workspace = true } rand = { workspace = true } @@ -123,8 +123,8 @@ sqlx = { workspace = true, features = ["runtime-tokio", "postgres"] } tempfile = { workspace = true } quickwit-actors = { workspace = true, features = ["testsuite"] } -quickwit-parquet-engine = { workspace = true, features = ["testsuite"] } quickwit-cluster = { workspace = true, features = ["testsuite"] } +quickwit-parquet-engine = { workspace = true, features = ["testsuite"] } quickwit-common = { workspace = true, features = ["testsuite"] } quickwit-config = { workspace = true, features = ["testsuite"] } quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index 84ba3987f4a..3d272caf11a 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -49,8 +49,8 @@ use tokio::sync::Semaphore; use tracing::{Span, info, info_span, warn}; use ulid::Ulid; -use crate::actors::IndexSerializer; -use crate::actors::cooperative_indexing::{CooperativeIndexingCycle, CooperativeIndexingPeriod}; +use super::IndexSerializer; +use super::cooperative_indexing::{CooperativeIndexingCycle, CooperativeIndexingPeriod}; use crate::models::{ CommitTrigger, EmptySplit, IndexedSplitBatchBuilder, IndexedSplitBuilder, NewPublishLock, NewPublishToken, ProcessedDoc, ProcessedDocBatch, PublishLock, @@ -715,8 +715,7 @@ mod tests { use quickwit_proto::types::{IndexUid, NodeId, PipelineUid}; use tantivy::{DateTime, doc}; - use super::*; - use crate::actors::indexer::{IndexerCounters, record_timestamp}; + use super::{IndexerCounters, record_timestamp, *}; #[test] fn test_record_timestamp() { diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index cfb2ade9361..93b28095f11 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -15,18 +15,17 @@ use std::collections::BTreeSet; use std::path::PathBuf; use std::sync::Arc; -use std::sync::atomic::Ordering; -use std::time::{Duration, Instant}; +use std::time::Instant; use async_trait::async_trait; use quickwit_actors::{ Actor, ActorContext, ActorExitStatus, ActorHandle, HEARTBEAT, Handler, Health, Mailbox, QueueCapacity, Supervisable, }; +use quickwit_common::KillSwitch; use quickwit_common::metrics::OwnedGaugeGuard; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; -use quickwit_common::{KillSwitch, is_metrics_index}; use quickwit_config::{IndexingSettings, RetentionPolicy, SourceConfig}; use quickwit_doc_mapper::DocMapper; use quickwit_ingest::IngesterPool; @@ -37,56 +36,26 @@ use quickwit_storage::{Storage, StorageResolver}; use tokio::sync::Semaphore; use tracing::{debug, error, info, instrument}; -use super::MergePlanner; +use super::{DocProcessor, IndexSerializer, Indexer, MergePlanner, Packager}; use crate::SplitsUpdateMailbox; -use crate::actors::doc_processor::DocProcessor; -use crate::actors::index_serializer::IndexSerializer; -use crate::actors::publisher::PublisherType; +use crate::actors::pipeline_shared::{ + SPAWN_PIPELINE_SEMAPHORE, SUPERVISE_INTERVAL, Spawn, SuperviseLoop, wait_duration_before_retry, +}; use crate::actors::sequencer::Sequencer; use crate::actors::uploader::UploaderType; -use crate::actors::{ - Indexer, Packager, ParquetDocProcessor, ParquetIndexer, ParquetPackager, ParquetPublisher, - ParquetUploader, Publisher, Uploader, -}; +use crate::actors::{Publisher, Uploader}; use crate::merge_policy::MergePolicy; use crate::models::IndexingStatistics; use crate::source::{ - AssignShards, Assignment, SourceActor, SourceRuntime, quickwit_supported_parquet_sources, + AssignShards, Assignment, ProcessorMailbox, SourceActor, SourceRuntime, quickwit_supported_sources, }; use crate::split_store::IndexingSplitStore; -const SUPERVISE_INTERVAL: Duration = Duration::from_secs(1); - -const MAX_RETRY_DELAY: Duration = Duration::from_secs(600); // 10 min. - -#[derive(Debug)] -struct SuperviseLoop; - -/// Calculates the wait time based on retry count. -// retry_count, wait_time -// 0 1s -// 1 2s -// 2 4s -// 3 8s -// ... -// >=8 5mn -pub(crate) fn wait_duration_before_retry(retry_count: usize) -> Duration { - // Protect against a `retry_count` that will lead to an overflow. - let max_power = (retry_count as u32).min(31); - Duration::from_secs(2u64.pow(max_power)).min(MAX_RETRY_DELAY) -} - -/// Spawning an indexing pipeline puts a lot of pressure on the file system, metastore, etc. so -/// we rely on this semaphore to limit the number of indexing pipelines that can be spawned -/// concurrently. -/// See also . -static SPAWN_PIPELINE_SEMAPHORE: Semaphore = Semaphore::const_new(10); - /// Handles for standard Tantivy-based indexing pipeline. struct IndexingPipelineHandles { - source_mailbox: Mailbox>, - source_handle: ActorHandle>, + source_mailbox: Mailbox, + source_handle: ActorHandle, doc_processor: ActorHandle, indexer: ActorHandle, index_serializer: ActorHandle, @@ -108,54 +77,11 @@ impl IndexingPipelineHandles { } } -/// Handles for metrics Parquet/DataFusion indexing pipeline. -struct ParquetIndexingPipelineHandles { - source_mailbox: Mailbox>, - source_handle: ActorHandle>, - doc_processor: ActorHandle, - indexer: ActorHandle, - packager: ActorHandle, - uploader: ActorHandle, - sequencer: ActorHandle>, - publisher: ActorHandle, - next_check_for_progress: Instant, -} - -impl ParquetIndexingPipelineHandles { - fn should_check_for_progress(&mut self) -> bool { - let now = Instant::now(); - let check_for_progress = now > self.next_check_for_progress; - if check_for_progress { - self.next_check_for_progress = now + *HEARTBEAT; - } - check_for_progress - } -} - -/// Enum representing either a standard or metrics pipeline's actor handles. -/// -/// This allows `IndexingPipeline` to properly manage both pipeline types -/// without resorting to `Option` workarounds where `None` implicitly means -/// "metrics pipeline". -enum PipelineHandles { - /// Standard Tantivy-based pipeline for logs/traces. - Standard(IndexingPipelineHandles), - /// Parquet/DataFusion pipeline for time-series data. - Parquet(ParquetIndexingPipelineHandles), -} - -// Messages - -#[derive(Clone, Copy, Debug, Default)] -pub struct Spawn { - retry_count: usize, -} - pub struct IndexingPipeline { params: IndexingPipelineParams, previous_generations_statistics: IndexingStatistics, statistics: IndexingStatistics, - handles_opt: Option, + handles_opt: Option, // Killswitch used for the actors in the pipeline. This is not the supervisor killswitch. kill_switch: KillSwitch, @@ -219,7 +145,7 @@ impl IndexingPipeline { fn supervisables(&self) -> Vec<&dyn Supervisable> { match &self.handles_opt { - Some(PipelineHandles::Standard(handles)) => { + Some(handles) => { vec![ &handles.source_handle, &handles.doc_processor, @@ -231,17 +157,6 @@ impl IndexingPipeline { &handles.publisher, ] } - Some(PipelineHandles::Parquet(handles)) => { - vec![ - &handles.source_handle, - &handles.doc_processor, - &handles.indexer, - &handles.packager, - &handles.uploader, - &handles.sequencer, - &handles.publisher, - ] - } None => Vec::new(), } } @@ -304,57 +219,24 @@ impl IndexingPipeline { } fn perform_observe(&mut self, ctx: &ActorContext) { - // Update statistics based on pipeline type - match &self.handles_opt { - Some(PipelineHandles::Standard(handles)) => { - handles.doc_processor.refresh_observe(); - handles.indexer.refresh_observe(); - handles.uploader.refresh_observe(); - handles.publisher.refresh_observe(); - self.statistics = self - .previous_generations_statistics - .clone() - .add_actor_counters( - &handles.doc_processor.last_observation(), - &handles.indexer.last_observation(), - &handles.uploader.last_observation(), - &handles.publisher.last_observation(), - ) - .set_generation(self.statistics.generation) - .set_num_spawn_attempts(self.statistics.num_spawn_attempts); - let pipeline_metrics_opt = handles.indexer.last_observation().pipeline_metrics_opt; - self.statistics.pipeline_metrics_opt = pipeline_metrics_opt; - } - Some(PipelineHandles::Parquet(handles)) => { - handles.doc_processor.refresh_observe(); - handles.indexer.refresh_observe(); - handles.uploader.refresh_observe(); - handles.publisher.refresh_observe(); - - let doc_counters = handles.doc_processor.last_observation(); - let indexer_counters = handles.indexer.last_observation(); - let uploader_counters = handles.uploader.last_observation(); - let publisher_counters = handles.publisher.last_observation(); - - let mut stats = self.previous_generations_statistics.clone(); - stats.num_docs += doc_counters.valid_rows; - stats.num_invalid_docs += doc_counters.num_errors(); - stats.total_bytes_processed += doc_counters.bytes_total; - stats.num_local_splits += indexer_counters.batches_flushed; - stats.num_staged_splits += - uploader_counters.num_staged_splits.load(Ordering::Relaxed); - stats.num_uploaded_splits += uploader_counters - .num_uploaded_splits - .load(Ordering::Relaxed); - stats.num_published_splits += publisher_counters.num_published_splits; - stats.num_empty_splits += publisher_counters.num_empty_splits; - stats.generation = self.statistics.generation; - stats.num_spawn_attempts = self.statistics.num_spawn_attempts; - self.statistics = stats; - } - None => { - // Pipeline not yet spawned - } + if let Some(handles) = &self.handles_opt { + handles.doc_processor.refresh_observe(); + handles.indexer.refresh_observe(); + handles.uploader.refresh_observe(); + handles.publisher.refresh_observe(); + self.statistics = self + .previous_generations_statistics + .clone() + .add_actor_counters( + &handles.doc_processor.last_observation(), + &handles.indexer.last_observation(), + &handles.uploader.last_observation(), + &handles.publisher.last_observation(), + ) + .set_generation(self.statistics.generation) + .set_num_spawn_attempts(self.statistics.num_spawn_attempts); + let pipeline_metrics_opt = handles.indexer.last_observation().pipeline_metrics_opt; + self.statistics.pipeline_metrics_opt = pipeline_metrics_opt; } // Always update params_fingerprint, shard_ids, and emit observation. // This ensures shard assignments are reported to the control plane via chitchat. @@ -368,10 +250,8 @@ impl IndexingPipeline { &mut self, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - // Check if it's time to verify progress based on pipeline type let check_for_progress = match &mut self.handles_opt { - Some(PipelineHandles::Standard(handles)) => handles.should_check_for_progress(), - Some(PipelineHandles::Parquet(handles)) => handles.should_check_for_progress(), + Some(handles) => handles.should_check_for_progress(), None => return Ok(()), }; let health = self.healthcheck(check_for_progress); @@ -408,12 +288,6 @@ impl IndexingPipeline { self.kill_switch = ctx.kill_switch().child(); let index_id = &self.params.pipeline_id.index_uid.index_id; - - // Route metrics indexes to the Parquet/DataFusion pipeline - if is_metrics_index(index_id) { - return self.spawn_parquet_pipeline(ctx).await; - } - let source_id = &self.params.pipeline_id.source_id; info!( @@ -425,11 +299,12 @@ impl IndexingPipeline { ); let (source_mailbox, source_inbox) = ctx .spawn_ctx() - .create_mailbox::>("SourceActor", QueueCapacity::Unbounded); + .create_mailbox::("SourceActor", QueueCapacity::Unbounded); // Publisher let publisher = Publisher::new( - PublisherType::MainPublisher, + super::PUBLISHER_NAME, + QueueCapacity::Bounded(1), self.params.metastore.clone(), Some(self.params.merge_planner_mailbox.clone()), Some(source_mailbox.clone()), @@ -543,7 +418,7 @@ impl IndexingPipeline { .await?; let actor_source = SourceActor { source, - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (source_mailbox, source_handle) = ctx .spawn_actor() @@ -558,7 +433,7 @@ impl IndexingPipeline { // Increment generation once we are sure there will be no spawning error. self.previous_generations_statistics = self.statistics.clone(); self.statistics.generation += 1; - self.handles_opt = Some(PipelineHandles::Standard(IndexingPipelineHandles { + self.handles_opt = Some(IndexingPipelineHandles { source_mailbox, source_handle, doc_processor: doc_processor_handle, @@ -569,179 +444,20 @@ impl IndexingPipeline { sequencer: sequencer_handle, publisher: publisher_handle, next_check_for_progress: Instant::now() + *HEARTBEAT, - })); - Ok(()) - } - - /// Spawn the parquet pipeline using ParquetDocProcessor and ParquetIndexer. - /// - /// This pipeline routes metrics data through the Parquet/DataFusion path instead of - /// the Tantivy path. The output is ParquetSplit files written to the indexing directory. - #[instrument( - name="spawn_parquet_pipeline", - level="info", - skip_all, - fields( - index=%self.params.pipeline_id.index_uid.index_id, - r#gen=self.generation() - ))] - async fn spawn_parquet_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { - let index_id = &self.params.pipeline_id.index_uid.index_id; - let source_id = &self.params.pipeline_id.source_id; - - info!( - index_id, - source_id, - pipeline_uid=%self.params.pipeline_id.pipeline_uid, - root_dir=%self.params.indexing_directory.path().display(), - "spawning parquet indexing pipeline for metrics", - ); - - let (source_mailbox, source_inbox) = ctx - .spawn_ctx() - .create_mailbox::>( - "SourceActor", - QueueCapacity::Unbounded, - ); - - // ParquetPublisher - let parquet_publisher = ParquetPublisher::new( - PublisherType::ParquetPublisher, - self.params.metastore.clone(), - None, - Some(source_mailbox.clone()), - ); - let (parquet_publisher_mailbox, parquet_publisher_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_publisher); - - // Sequencer for ordered delivery - let parquet_sequencer = Sequencer::new(parquet_publisher_mailbox); - let (parquet_sequencer_mailbox, parquet_sequencer_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_sequencer); - - // ParquetUploader - let parquet_uploader = ParquetUploader::new( - super::UploaderType::IndexUploader, - self.params.metastore.clone(), - self.params.storage.clone(), - SplitsUpdateMailbox::Sequencer(parquet_sequencer_mailbox), - self.params.max_concurrent_split_uploads_index, - ); - let (parquet_uploader_mailbox, parquet_uploader_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_uploader); - - // ParquetPackager - let writer_config = quickwit_parquet_engine::storage::ParquetWriterConfig::default(); - let split_writer = quickwit_parquet_engine::storage::ParquetSplitWriter::new( - writer_config, - self.params.indexing_directory.path(), - ); - let parquet_packager = ParquetPackager::new(split_writer, parquet_uploader_mailbox); - let (parquet_packager_mailbox, parquet_packager_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_packager); - - // ParquetIndexer - let commit_timeout = - Duration::from_secs(self.params.indexing_settings.commit_timeout_secs as u64); - let parquet_indexer = ParquetIndexer::new( - self.params.pipeline_id.index_uid.clone(), - source_id.to_string(), - None, - parquet_packager_mailbox, - Some(commit_timeout), - ); - let (parquet_indexer_mailbox, parquet_indexer_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_indexer); - - // ParquetDocProcessor - let parquet_doc_processor = ParquetDocProcessor::new( - index_id.to_string(), - source_id.to_string(), - parquet_indexer_mailbox, - ); - let (parquet_doc_processor_mailbox, parquet_doc_processor_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) - .spawn(parquet_doc_processor); - - // Source (using parquet source loader) - let source_runtime = SourceRuntime { - pipeline_id: self.params.pipeline_id.clone(), - source_config: self.params.source_config.clone(), - metastore: self.params.metastore.clone(), - ingester_pool: self.params.ingester_pool.clone(), - queues_dir_path: self.params.queues_dir_path.clone(), - storage_resolver: self.params.source_storage_resolver.clone(), - event_broker: self.params.event_broker.clone(), - indexing_setting: self.params.indexing_settings.clone(), - }; - let source = ctx - .protect_future(quickwit_supported_parquet_sources().load_source(source_runtime)) - .await?; - let actor_source = SourceActor { - source, - processor_mailbox: parquet_doc_processor_mailbox, - }; - let (source_mailbox, source_handle) = ctx - .spawn_actor() - .set_mailboxes(source_mailbox, source_inbox) - .set_kill_switch(self.kill_switch.clone()) - .spawn(actor_source); - let assign_shards_message = AssignShards(Assignment { - shard_ids: self.shard_ids.clone(), }); - source_mailbox.send_message(assign_shards_message).await?; - - // Increment generation once we are sure there will be no spawning error. - self.previous_generations_statistics = self.statistics.clone(); - self.statistics.generation += 1; - self.handles_opt = Some(PipelineHandles::Parquet(ParquetIndexingPipelineHandles { - source_mailbox, - source_handle, - doc_processor: parquet_doc_processor_handle, - indexer: parquet_indexer_handle, - packager: parquet_packager_handle, - uploader: parquet_uploader_handle, - sequencer: parquet_sequencer_handle, - publisher: parquet_publisher_handle, - next_check_for_progress: Instant::now() + *HEARTBEAT, - })); Ok(()) } async fn terminate(&mut self) { self.kill_switch.kill(); if let Some(handles) = self.handles_opt.take() { - match handles { - PipelineHandles::Standard(handles) => { - tokio::join!( - handles.source_handle.kill(), - handles.indexer.kill(), - handles.packager.kill(), - handles.uploader.kill(), - handles.publisher.kill(), - ); - } - PipelineHandles::Parquet(handles) => { - tokio::join!( - handles.source_handle.kill(), - handles.indexer.kill(), - handles.packager.kill(), - handles.uploader.kill(), - handles.publisher.kill(), - ); - } - } + tokio::join!( + handles.source_handle.kill(), + handles.indexer.kill(), + handles.packager.kill(), + handles.uploader.kill(), + handles.publisher.kill(), + ); } } } @@ -807,28 +523,15 @@ impl Handler for IndexingPipeline { .clone_from(&assign_shards_message.0.shard_ids); // If the pipeline is running, we forward the message to its source. // If it is not, it will be respawned soon, and the shards will be assigned afterward. - match &mut self.handles_opt { - Some(PipelineHandles::Standard(handles)) => { - info!( - shard_ids=?assign_shards_message.0.shard_ids, - "assigning shards to indexing pipeline" - ); - handles - .source_mailbox - .send_message(assign_shards_message) - .await?; - } - Some(PipelineHandles::Parquet(handles)) => { - info!( - shard_ids=?assign_shards_message.0.shard_ids, - "assigning shards to parquet indexing pipeline" - ); - handles - .source_mailbox - .send_message(assign_shards_message) - .await?; - } - None => {} + if let Some(handles) = &self.handles_opt { + info!( + shard_ids=?assign_shards_message.0.shard_ids, + "assigning shards to indexing pipeline" + ); + handles + .source_mailbox + .send_message(assign_shards_message) + .await?; } // We perform observe to make sure the set of shard ids is up to date. self.perform_observe(ctx); @@ -870,6 +573,7 @@ mod tests { use std::num::NonZeroUsize; use std::path::PathBuf; use std::sync::Arc; + use std::time::Duration; use quickwit_actors::{Command, Universe}; use quickwit_common::ServiceStream; @@ -885,7 +589,7 @@ mod tests { use quickwit_storage::RamStorage; use super::{IndexingPipeline, *}; - use crate::actors::merge_pipeline::{MergePipeline, MergePipelineParams}; + use crate::actors::{MergePipeline, MergePipelineParams}; use crate::merge_policy::default_merge_policy; #[test] @@ -895,7 +599,7 @@ mod tests { assert_eq!(wait_duration_before_retry(2), Duration::from_secs(4)); assert_eq!(wait_duration_before_retry(3), Duration::from_secs(8)); assert_eq!(wait_duration_before_retry(9), Duration::from_secs(512)); - assert_eq!(wait_duration_before_retry(10), MAX_RETRY_DELAY); + assert_eq!(wait_duration_before_retry(10), Duration::from_secs(600)); } async fn test_indexing_pipeline_num_fails_before_success( diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 5d307fb200b..f700c4f1917 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -28,6 +28,8 @@ use quickwit_actors::{ use quickwit_cluster::Cluster; use quickwit_common::fs::get_cache_directory_path; use quickwit_common::io::Limiter; +#[cfg(feature = "metrics")] +use quickwit_common::is_metrics_index; use quickwit_common::pubsub::EventBroker; use quickwit_common::{io, temp_dir}; use quickwit_config::{ @@ -60,8 +62,8 @@ use tokio::sync::Semaphore; use tracing::{debug, error, info, warn}; use super::merge_pipeline::{MergePipeline, MergePipelineParams}; -use super::{MergePlanner, MergeSchedulerService}; -use crate::actors::merge_pipeline::FinishPendingMergesAndShutdownPipeline; +use super::pipeline_shared::{ActorPipeline, PipelineHandle}; +use super::{FinishPendingMergesAndShutdownPipeline, MergePlanner, MergeSchedulerService}; use crate::models::{DetachIndexingPipeline, DetachMergePipeline, ObservePipeline, SpawnPipeline}; use crate::source::{AssignShards, Assignment}; use crate::split_store::{IndexingSplitCache, SplitStoreQuota}; @@ -85,11 +87,7 @@ struct MergePipelineHandle { handle: ActorHandle, } -struct PipelineHandle { - mailbox: Mailbox, - handle: ActorHandle, - indexing_pipeline_id: IndexingPipelineId, -} +pub type BoxPipelineHandle = Box; /// The indexing service is (single) actor service running on indexer and in charge /// of executing the indexing plans received from the control plane. @@ -99,22 +97,22 @@ struct PipelineHandle { /// are respectively missing or extranumerous. pub struct IndexingService { node_id: NodeId, - indexing_root_directory: PathBuf, - queue_dir_path: PathBuf, + pub(crate) indexing_root_directory: PathBuf, + pub(crate) queue_dir_path: PathBuf, cluster: Cluster, - metastore: MetastoreServiceClient, + pub(crate) metastore: MetastoreServiceClient, ingest_api_service_opt: Option>, merge_scheduler_service: Mailbox, - ingester_pool: IngesterPool, - storage_resolver: StorageResolver, - indexing_pipelines: HashMap, + pub(crate) ingester_pool: IngesterPool, + pub(crate) storage_resolver: StorageResolver, + indexing_pipelines: HashMap, counters: IndexingServiceCounters, local_split_store: Arc, - max_concurrent_split_uploads: usize, + pub(crate) max_concurrent_split_uploads: usize, merge_pipeline_handles: HashMap, cooperative_indexing_permits: Option>, merge_io_throughput_limiter_opt: Option, - event_broker: EventBroker, + pub(crate) event_broker: EventBroker, } impl Debug for IndexingService { @@ -184,7 +182,7 @@ impl IndexingService { async fn detach_indexing_pipeline( &mut self, pipeline_uid: &PipelineUid, - ) -> Result, IndexingError> { + ) -> Result { let pipeline_handle = self .indexing_pipelines .remove(pipeline_uid) @@ -193,7 +191,7 @@ impl IndexingService { IndexingError::Internal(message) })?; self.counters.num_running_pipelines -= 1; - Ok(pipeline_handle.handle) + Ok(pipeline_handle) } async fn detach_merge_pipeline( @@ -215,14 +213,10 @@ impl IndexingService { &mut self, pipeline_uid: &PipelineUid, ) -> Result, IndexingError> { - let pipeline_handle = &self - .indexing_pipelines - .get(pipeline_uid) - .ok_or_else(|| { - let message = format!("could not find indexing pipeline `{pipeline_uid}`"); - IndexingError::Internal(message) - })? - .handle; + let pipeline_handle = self.indexing_pipelines.get(pipeline_uid).ok_or_else(|| { + let message = format!("could not find indexing pipeline `{pipeline_uid}`"); + IndexingError::Internal(message) + })?; let observation = pipeline_handle.observe().await; Ok(observation) } @@ -270,6 +264,69 @@ impl IndexingService { let message = format!("pipeline `{indexing_pipeline_id}` already exists"); return Err(IndexingError::Internal(message)); } + + let params_fingerprint = + indexing_pipeline_params_fingerprint(&index_config, &source_config); + if let Some(expected_params_fingerprint) = expected_params_fingerprint { + if params_fingerprint != expected_params_fingerprint { + info!( + index_id = indexing_pipeline_id.index_uid.index_id, + source_id = indexing_pipeline_id.source_id, + expected = expected_params_fingerprint, + actual = params_fingerprint, + "pipeline fingerprint mismatch, postponing pipeline creation" + ); + return Ok(()); + } + } + + #[cfg(feature = "metrics")] + let pipeline_handle = if is_metrics_index(&indexing_pipeline_id.index_uid.index_id) { + self.spawn_metrics_pipeline( + ctx, + indexing_pipeline_id.clone(), + index_config, + source_config, + params_fingerprint, + ) + .await? + } else { + self.spawn_log_pipeline( + ctx, + indexing_pipeline_id.clone(), + index_config, + source_config, + immature_splits_opt, + params_fingerprint, + ) + .await? + }; + #[cfg(not(feature = "metrics"))] + let pipeline_handle = self + .spawn_log_pipeline( + ctx, + indexing_pipeline_id.clone(), + index_config, + source_config, + immature_splits_opt, + params_fingerprint, + ) + .await?; + self.indexing_pipelines + .insert(indexing_pipeline_id.pipeline_uid, pipeline_handle); + self.counters.num_running_pipelines += 1; + Ok(()) + } + + async fn spawn_log_pipeline( + &mut self, + ctx: &ActorContext, + indexing_pipeline_id: IndexingPipelineId, + index_config: IndexConfig, + source_config: SourceConfig, + immature_splits_opt: Option>, + params_fingerprint: u64, + ) -> Result { let pipeline_uid_str = indexing_pipeline_id.pipeline_uid.to_string(); let indexing_directory = temp_dir::Builder::default() .join(&indexing_pipeline_id.index_uid.index_id) @@ -319,63 +376,34 @@ impl IndexingService { let max_concurrent_split_uploads_merge = (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); - let params_fingerprint = - indexing_pipeline_params_fingerprint(&index_config, &source_config); - if let Some(expected_params_fingerprint) = expected_params_fingerprint { - // If the fingerprint of the config freshly fetched from the - // metastore is different from that received from the control plane, - // it means that the config changed again since the last indexing - // plan was built. In this case, postpone the pipeline creation. - if params_fingerprint != expected_params_fingerprint { - info!( - index_id = indexing_pipeline_id.index_uid.index_id, - source_id = indexing_pipeline_id.source_id, - expected = expected_params_fingerprint, - actual = params_fingerprint, - "pipeline fingerprint mismatch, postponing pipeline creation" - ); - return Ok(()); - } - } let pipeline_params = IndexingPipelineParams { pipeline_id: indexing_pipeline_id.clone(), metastore: self.metastore.clone(), storage, - - // Indexing-related parameters doc_mapper, indexing_directory, indexing_settings: index_config.indexing_settings.clone(), split_store, max_concurrent_split_uploads_index, cooperative_indexing_permits: self.cooperative_indexing_permits.clone(), - - // Merge-related parameters merge_policy, retention_policy, max_concurrent_split_uploads_merge, merge_planner_mailbox, - - // Source-related parameters source_config, ingester_pool: self.ingester_pool.clone(), queues_dir_path: self.queue_dir_path.clone(), source_storage_resolver: self.storage_resolver.clone(), params_fingerprint, - event_broker: self.event_broker.clone(), }; let pipeline = IndexingPipeline::new(pipeline_params); - let (pipeline_mailbox, pipeline_handle) = ctx.spawn_actor().spawn(pipeline); - let pipeline_handle = PipelineHandle { - mailbox: pipeline_mailbox, - handle: pipeline_handle, - indexing_pipeline_id: indexing_pipeline_id.clone(), - }; - self.indexing_pipelines - .insert(indexing_pipeline_id.pipeline_uid, pipeline_handle); - self.counters.num_running_pipelines += 1; - Ok(()) + let (mailbox, handle) = ctx.spawn_actor().spawn(pipeline); + Ok(Box::new(ActorPipeline { + pipeline_id: indexing_pipeline_id, + mailbox, + handle, + })) } async fn index_metadata( @@ -485,7 +513,7 @@ impl IndexingService { async fn handle_supervise(&mut self) -> Result<(), ActorExitStatus> { self.indexing_pipelines .retain(|pipeline_uid, pipeline_handle| { - match pipeline_handle.handle.state() { + match pipeline_handle.state() { ActorState::Paused | ActorState::Running => true, ActorState::Success => { info!( @@ -512,7 +540,7 @@ impl IndexingService { let merge_pipelines_to_retain: HashSet = self .indexing_pipelines .values() - .map(|pipeline_handle| pipeline_handle.indexing_pipeline_id.merge_pipeline_id()) + .map(|pipeline_handle| pipeline_handle.indexing_pipeline_id().merge_pipeline_id()) .collect(); let merge_pipelines_to_shutdown: Vec = self @@ -557,9 +585,9 @@ impl IndexingService { .indexing_pipelines .values() .filter_map(|pipeline_handle| { - let indexing_statistics = pipeline_handle.handle.last_observation(); + let indexing_statistics = pipeline_handle.last_observation(); let pipeline_metrics = indexing_statistics.pipeline_metrics_opt?; - Some((&pipeline_handle.indexing_pipeline_id, pipeline_metrics)) + Some((pipeline_handle.indexing_pipeline_id(), pipeline_metrics)) }) .collect(); self.cluster @@ -614,7 +642,7 @@ impl IndexingService { }; let message = AssignShards(assignment); - if let Err(error) = pipeline_handle.mailbox.send_message(message).await { + if let Err(error) = pipeline_handle.send_assign_shards(message).await { error!(%error, "failed to assign shards to indexing pipeline"); } } @@ -755,7 +783,7 @@ impl IndexingService { .iter() .flat_map(|pipeline_uid| self.indexing_pipelines.get(pipeline_uid)) .any(|pipeline_handle| { - pipeline_handle.indexing_pipeline_id.source_id == INGEST_API_SOURCE_ID + pipeline_handle.indexing_pipeline_id().source_id == INGEST_API_SOURCE_ID }); for pipeline_to_shutdown in pipelines_to_shutdown { @@ -792,12 +820,12 @@ impl IndexingService { .indexing_pipelines .values() .map(|pipeline_handle| { - let assignment = pipeline_handle.handle.last_observation(); + let assignment = pipeline_handle.last_observation(); let shard_ids: Vec = assignment.shard_ids.iter().cloned().collect(); IndexingTask { - index_uid: Some(pipeline_handle.indexing_pipeline_id.index_uid.clone()), - source_id: pipeline_handle.indexing_pipeline_id.source_id.clone(), - pipeline_uid: Some(pipeline_handle.indexing_pipeline_id.pipeline_uid), + index_uid: Some(pipeline_handle.indexing_pipeline_id().index_uid.clone()), + source_id: pipeline_handle.indexing_pipeline_id().source_id.clone(), + pipeline_uid: Some(pipeline_handle.indexing_pipeline_id().pipeline_uid), shard_ids, params_fingerprint: assignment.params_fingerprint, } @@ -888,7 +916,7 @@ impl Handler for IndexingService { #[async_trait] impl Handler for IndexingService { - type Reply = Result, IndexingError>; + type Reply = Result; async fn handle( &mut self, @@ -1007,7 +1035,7 @@ mod tests { use std::path::Path; use std::time::Duration; - use quickwit_actors::{HEARTBEAT, Health, ObservationType, Supervisable, Universe}; + use quickwit_actors::{HEARTBEAT, Health, ObservationType, Universe}; use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; use quickwit_common::ServiceStream; use quickwit_common::rand::append_random_suffix; @@ -1639,7 +1667,6 @@ mod tests { .indexing_pipelines .get(&message.0.pipeline_uid) .unwrap() - .handle .check_health(true)) } } diff --git a/quickwit/quickwit-indexing/src/actors/log_publisher_impl.rs b/quickwit/quickwit-indexing/src/actors/log_publisher_impl.rs new file mode 100644 index 00000000000..ba35cb97620 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/log_publisher_impl.rs @@ -0,0 +1,378 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! `Handler` and `Handler` implementations +//! for `Publisher`, specific to the logs/traces pipeline. + +use anyhow::Context; +use async_trait::async_trait; +use fail::fail_point; +use quickwit_actors::{ActorContext, ActorExitStatus, Handler}; +use quickwit_proto::metastore::{MetastoreService, PublishSplitsRequest}; +use tracing::{info, instrument}; + +use crate::actors::publisher::{ + DisconnectMergePlanner, Publisher, serialize_checkpoint_delta, suggest_truncate, +}; +use crate::models::{NewSplits, SplitsUpdate}; + +pub(crate) const PUBLISHER_NAME: &str = "Publisher"; +pub(crate) const MERGE_PUBLISHER_NAME: &str = "MergePublisher"; + +#[async_trait] +impl Handler for Publisher { + type Reply = (); + + async fn handle( + &mut self, + _: DisconnectMergePlanner, + _ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + info!("disconnecting merge planner mailbox"); + self.merge_planner_mailbox_opt = None; + Ok(()) + } +} + +#[async_trait] +impl Handler for Publisher { + type Reply = (); + + #[instrument(name="publisher", parent=split_update.parent_span.id(), skip(self, ctx))] + async fn handle( + &mut self, + split_update: SplitsUpdate, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + fail_point!("publisher:before"); + + let SplitsUpdate { + index_uid, + new_splits, + replaced_split_ids, + checkpoint_delta_opt, + publish_lock, + publish_token_opt, + .. + } = split_update; + + let index_checkpoint_delta_json_opt = serialize_checkpoint_delta(&checkpoint_delta_opt)?; + let split_ids: Vec = new_splits + .iter() + .map(|split| split.split_id.clone()) + .collect(); + if let Some(_guard) = publish_lock.acquire().await { + let publish_splits_request = PublishSplitsRequest { + index_uid: Some(index_uid), + staged_split_ids: split_ids.clone(), + replaced_split_ids: replaced_split_ids.clone(), + index_checkpoint_delta_json_opt, + publish_token_opt: publish_token_opt.clone(), + }; + ctx.protect_future(self.metastore.publish_splits(publish_splits_request)) + .await + .context("failed to publish splits")?; + } else { + info!( + split_ids=?split_ids, + "Splits' publish lock is dead." + ); + return Ok(()); + } + info!("publish-new-splits"); + suggest_truncate(ctx, &self.source_mailbox_opt, checkpoint_delta_opt).await; + + if !new_splits.is_empty() { + if let Some(merge_planner_mailbox) = self.merge_planner_mailbox_opt.as_ref() { + let _ = ctx + .send_message(merge_planner_mailbox, NewSplits { new_splits }) + .await; + } + + if replaced_split_ids.is_empty() { + self.counters.num_published_splits += 1; + } else { + self.counters.num_replace_operations += 1; + } + } else { + self.counters.num_empty_splits += 1; + } + fail_point!("publisher:after"); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use quickwit_actors::{QueueCapacity, Universe}; + use quickwit_metastore::checkpoint::{ + IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, + }; + use quickwit_metastore::{PublishSplitsRequestExt, SplitMetadata}; + use quickwit_proto::metastore::{EmptyResponse, MetastoreServiceClient, MockMetastoreService}; + use quickwit_proto::types::{IndexUid, Position}; + use tracing::Span; + + use super::PUBLISHER_NAME; + use crate::actors::publisher::Publisher; + use crate::models::{PublishLock, SplitsUpdate}; + use crate::source::SuggestTruncate; + + #[tokio::test] + async fn test_publisher_publish_operation() { + let universe = Universe::with_accelerated_time(); + let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); + let mut mock_metastore = MockMetastoreService::new(); + let ref_index_uid_clone = ref_index_uid.clone(); + mock_metastore + .expect_publish_splits() + .withf(move |publish_splits_request| { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid() == &ref_index_uid_clone + && checkpoint_delta.source_id == "source" + && publish_splits_request.staged_split_ids[..] == ["split"] + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) + }) + .times(1) + .returning(|_| Ok(EmptyResponse {})); + let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); + + let (source_mailbox, source_inbox) = universe.create_test_mailbox(); + + let publisher = Publisher::new( + PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + Some(merge_planner_mailbox), + Some(source_mailbox), + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + assert!( + publisher_mailbox + .send_message(SplitsUpdate { + index_uid: ref_index_uid.clone(), + new_splits: vec![SplitMetadata { + split_id: "split".to_string(), + ..Default::default() + }], + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: Some(IndexCheckpointDelta { + source_id: "source".to_string(), + source_delta: SourceCheckpointDelta::from_range(1..3), + }), + publish_lock: PublishLock::default(), + publish_token_opt: None, + merge_task: None, + parent_span: tracing::Span::none(), + }) + .await + .is_ok() + ); + + let publisher_observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(publisher_observation.num_published_splits, 1); + + let suggest_truncate_checkpoints: Vec = source_inbox + .drain_for_test_typed::() + .into_iter() + .map(|msg| msg.0) + .collect(); + + assert_eq!(suggest_truncate_checkpoints.len(), 1); + assert_eq!( + suggest_truncate_checkpoints[0] + .position_for_partition(&PartitionId::default()) + .unwrap(), + &Position::offset(2u64) + ); + + use crate::models::NewSplits; + let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); + assert_eq!(merger_msgs.len(), 1); + assert_eq!(merger_msgs[0].new_splits.len(), 1); + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_publisher_publish_operation_with_empty_splits() { + let universe = Universe::with_accelerated_time(); + let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); + let mut mock_metastore = MockMetastoreService::new(); + let ref_index_uid_clone = ref_index_uid.clone(); + mock_metastore + .expect_publish_splits() + .withf(move |publish_splits_request| { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid() == &ref_index_uid_clone + && checkpoint_delta.source_id == "source" + && publish_splits_request.staged_split_ids.is_empty() + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) + }) + .times(1) + .returning(|_| Ok(EmptyResponse {})); + let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); + + let (source_mailbox, source_inbox) = universe.create_test_mailbox(); + + let publisher = Publisher::new( + PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + Some(merge_planner_mailbox), + Some(source_mailbox), + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + assert!( + publisher_mailbox + .send_message(SplitsUpdate { + index_uid: ref_index_uid.clone(), + new_splits: Vec::new(), + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: Some(IndexCheckpointDelta { + source_id: "source".to_string(), + source_delta: SourceCheckpointDelta::from_range(1..3), + }), + publish_lock: PublishLock::default(), + publish_token_opt: None, + merge_task: None, + parent_span: tracing::Span::none(), + }) + .await + .is_ok() + ); + + let publisher_observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(publisher_observation.num_published_splits, 0); + assert_eq!(publisher_observation.num_replace_operations, 0); + assert_eq!(publisher_observation.num_empty_splits, 1); + + let suggest_truncate_checkpoints: Vec = source_inbox + .drain_for_test_typed::() + .into_iter() + .map(|msg| msg.0) + .collect(); + + assert_eq!(suggest_truncate_checkpoints.len(), 1); + + use crate::models::NewSplits; + let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); + assert_eq!(merger_msgs.len(), 0); + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_publisher_replace_operation() { + let universe = Universe::with_accelerated_time(); + let mut mock_metastore = MockMetastoreService::new(); + let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); + let ref_index_uid_clone = ref_index_uid.clone(); + mock_metastore + .expect_publish_splits() + .withf(move |publish_splits_requests| { + publish_splits_requests.index_uid() == &ref_index_uid_clone + && publish_splits_requests.staged_split_ids[..] == ["split3"] + && publish_splits_requests.replaced_split_ids[..] == ["split1", "split2"] + && publish_splits_requests + .index_checkpoint_delta_json_opt() + .is_empty() + }) + .times(1) + .returning(|_| Ok(EmptyResponse {})); + let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); + let publisher = Publisher::new( + PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + Some(merge_planner_mailbox), + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + publisher_mailbox + .send_message(SplitsUpdate { + index_uid: ref_index_uid.clone(), + new_splits: vec![SplitMetadata { + split_id: "split3".to_string(), + ..Default::default() + }], + replaced_split_ids: vec!["split1".to_string(), "split2".to_string()], + checkpoint_delta_opt: None, + publish_lock: PublishLock::default(), + publish_token_opt: None, + merge_task: None, + parent_span: Span::none(), + }) + .await + .unwrap(); + let publisher_observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(publisher_observation.num_published_splits, 0); + assert_eq!(publisher_observation.num_replace_operations, 1); + + use crate::models::NewSplits; + let merge_planner_msgs = merge_planner_inbox.drain_for_test_typed::(); + assert_eq!(merge_planner_msgs.len(), 1); + assert_eq!(merge_planner_msgs[0].new_splits.len(), 1); + universe.assert_quit().await; + } + + #[tokio::test] + async fn publisher_acquires_publish_lock() { + let universe = Universe::with_accelerated_time(); + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore.expect_publish_splits().never(); + let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); + + let publisher = Publisher::new( + PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + Some(merge_planner_mailbox), + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + let publish_lock = PublishLock::default(); + publish_lock.kill().await; + + publisher_mailbox + .send_message(SplitsUpdate { + index_uid: IndexUid::new_with_random_ulid("index"), + new_splits: vec![SplitMetadata::for_test("test-split".to_string())], + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: None, + publish_lock, + publish_token_opt: None, + merge_task: None, + parent_span: Span::none(), + }) + .await + .unwrap(); + + let publisher_observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(publisher_observation.num_published_splits, 0); + + let merger_messages = merge_planner_inbox.drain_for_test(); + assert!(merger_messages.is_empty()); + universe.assert_quit().await; + } +} diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index f871fbabb5e..b901d9f804a 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -18,7 +18,7 @@ use std::time::{Duration, Instant}; use async_trait::async_trait; use quickwit_actors::{ Actor, ActorContext, ActorExitStatus, ActorHandle, HEARTBEAT, Handler, Health, Inbox, Mailbox, - SpawnContext, Supervisable, + QueueCapacity, SpawnContext, Supervisable, }; use quickwit_common::KillSwitch; use quickwit_common::io::{IoControls, Limiter}; @@ -38,12 +38,11 @@ use time::OffsetDateTime; use tokio::sync::Semaphore; use tracing::{debug, error, info, instrument}; -use super::publisher::DisconnectMergePlanner; -use super::{MergeSchedulerService, RunFinalizeMergePolicyAndQuit}; -use crate::actors::indexing_pipeline::wait_duration_before_retry; -use crate::actors::merge_split_downloader::MergeSplitDownloader; -use crate::actors::publisher::PublisherType; -use crate::actors::{MergeExecutor, MergePlanner, Packager, Publisher, Uploader, UploaderType}; +use super::merge_planner::RunFinalizeMergePolicyAndQuit; +use super::{MergeExecutor, MergePlanner, MergeSplitDownloader, Packager}; +use crate::actors::pipeline_shared::wait_duration_before_retry; +use crate::actors::publisher::DisconnectMergePlanner; +use crate::actors::{MergeSchedulerService, Publisher, Uploader, UploaderType}; use crate::merge_policy::MergePolicy; use crate::models::MergeStatistics; use crate::split_store::IndexingSplitStore; @@ -264,7 +263,8 @@ impl MergePipeline { // Merge publisher let merge_publisher = Publisher::new( - PublisherType::MergePublisher, + super::MERGE_PUBLISHER_NAME, + QueueCapacity::Unbounded, self.params.metastore.clone(), Some(self.merge_planner_mailbox.clone()), None, @@ -592,8 +592,8 @@ mod tests { use quickwit_proto::types::{IndexUid, NodeId}; use quickwit_storage::RamStorage; + use super::{MergePipeline, MergePipelineParams}; use crate::IndexingSplitStore; - use crate::actors::merge_pipeline::{MergePipeline, MergePipelineParams}; use crate::actors::{MergePlanner, Publisher}; use crate::merge_policy::default_merge_policy; diff --git a/quickwit/quickwit-indexing/src/actors/merge_planner.rs b/quickwit/quickwit-indexing/src/actors/merge_planner.rs index 479cefcaa23..f807572c252 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_planner.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_planner.rs @@ -25,10 +25,9 @@ use tantivy::Inventory; use time::OffsetDateTime; use tracing::{info, warn}; -use super::MergeSchedulerService; +use super::merge_scheduler_service::schedule_merge; +use super::{MergeSchedulerService, MergeSplitDownloader}; use crate::MergePolicy; -use crate::actors::MergeSplitDownloader; -use crate::actors::merge_scheduler_service::schedule_merge; use crate::merge_policy::MergeOperation; use crate::models::NewSplits; diff --git a/quickwit/quickwit-indexing/src/actors/metrics_pipeline/indexing_service_impl.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/indexing_service_impl.rs new file mode 100644 index 00000000000..d5ce5a94e9c --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/indexing_service_impl.rs @@ -0,0 +1,61 @@ +use quickwit_actors::ActorContext; +use quickwit_common::temp_dir; +use quickwit_config::{IndexConfig, SourceConfig}; +use quickwit_proto::indexing::{IndexingError, IndexingPipelineId}; + +use crate::actors::pipeline_shared::ActorPipeline; +use crate::actors::{MetricsPipeline, MetricsPipelineParams}; +use crate::{BoxPipelineHandle, IndexingService}; + +impl IndexingService { + pub(crate) async fn spawn_metrics_pipeline( + &mut self, + ctx: &ActorContext, + indexing_pipeline_id: IndexingPipelineId, + index_config: IndexConfig, + source_config: SourceConfig, + params_fingerprint: u64, + ) -> Result { + let pipeline_uid_str = indexing_pipeline_id.pipeline_uid.to_string(); + let indexing_directory = temp_dir::Builder::default() + .join(&indexing_pipeline_id.index_uid.index_id) + .join(&indexing_pipeline_id.index_uid.incarnation_id.to_string()) + .join(&indexing_pipeline_id.source_id) + .join(&pipeline_uid_str) + .tempdir_in(&self.indexing_root_directory) + .map_err(|error| { + let message = format!("failed to create indexing directory: {error}"); + IndexingError::Internal(message) + })?; + let storage = self + .storage_resolver + .resolve(&index_config.index_uri) + .await + .map_err(|error| { + let message = format!("failed to spawn metrics pipeline: {error}"); + IndexingError::Internal(message) + })?; + + let pipeline_params = MetricsPipelineParams { + pipeline_id: indexing_pipeline_id.clone(), + metastore: self.metastore.clone(), + storage, + indexing_directory, + indexing_settings: index_config.indexing_settings.clone(), + max_concurrent_split_uploads: self.max_concurrent_split_uploads, + source_config, + ingester_pool: self.ingester_pool.clone(), + queues_dir_path: self.queue_dir_path.clone(), + source_storage_resolver: self.storage_resolver.clone(), + params_fingerprint, + event_broker: self.event_broker.clone(), + }; + let pipeline = MetricsPipeline::new(pipeline_params); + let (mailbox, handle) = ctx.spawn_actor().spawn(pipeline); + Ok(Box::new(ActorPipeline { + pipeline_id: indexing_pipeline_id, + mailbox, + handle, + })) + } +} diff --git a/quickwit/quickwit-indexing/src/actors/metrics_pipeline/mod.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/mod.rs new file mode 100644 index 00000000000..815b0ec3c18 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/mod.rs @@ -0,0 +1,52 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Metrics indexing pipeline actors. +//! +//! This module contains the Parquet/DataFusion-based pipeline for time-series +//! metrics data. The pipeline bypasses Tantivy and instead produces Parquet +//! split files: +//! +//! ```text +//! Source → ParquetDocProcessor → ParquetIndexer → ParquetPackager → ParquetUploader → Publisher +//! ``` + +mod indexing_service_impl; +mod parquet_doc_processor; +mod parquet_indexer; +mod parquet_packager; +mod parquet_splits_update; +mod parquet_uploader; +mod pipeline; +mod processed_parquet_batch; +mod publisher_impl; + +#[cfg(test)] +#[allow( + clippy::disallowed_methods, + clippy::needless_borrow, + clippy::unnecessary_map_or +)] +mod parquet_e2e_test; + +pub use parquet_doc_processor::{ + ParquetDocProcessor, ParquetDocProcessorCounters, ParquetDocProcessorError, is_arrow_ipc, +}; +pub use parquet_indexer::{ParquetIndexer, ParquetIndexerCounters, ParquetSplitBatch}; +pub use parquet_packager::{ParquetBatchForPackager, ParquetPackager, ParquetPackagerCounters}; +pub use parquet_splits_update::ParquetSplitsUpdate; +pub use parquet_uploader::ParquetUploader; +pub use pipeline::{MetricsPipeline, MetricsPipelineParams}; +pub use processed_parquet_batch::ProcessedParquetBatch; +pub(crate) use publisher_impl::METRICS_PUBLISHER_NAME; diff --git a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_doc_processor.rs similarity index 97% rename from quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_doc_processor.rs index eb51621a30f..eb32003a6af 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_doc_processor.rs @@ -29,10 +29,8 @@ use serde::Serialize; use tokio::runtime::Handle; use tracing::{debug, info, instrument}; -use super::ParquetIndexer; -use crate::models::{ - NewPublishLock, NewPublishToken, ProcessedParquetBatch, PublishLock, RawDocBatch, -}; +use super::{ParquetIndexer, ProcessedParquetBatch}; +use crate::models::{NewPublishLock, NewPublishToken, PublishLock, RawDocBatch}; /// Arrow IPC stream continuation marker (4 bytes of 0xFF). const ARROW_IPC_CONTINUATION_MARKER: [u8; 4] = [0xFF, 0xFF, 0xFF, 0xFF]; @@ -525,10 +523,8 @@ mod tests { use quickwit_proto::metastore::MockMetastoreService; use quickwit_storage::RamStorage; - use super::super::{ - ParquetIndexer, ParquetPackager, ParquetPublisher, ParquetUploader, SplitsUpdateMailbox, - }; - use crate::actors::UploaderType; + use crate::actors::metrics_pipeline::{ParquetIndexer, ParquetPackager, ParquetUploader}; + use crate::actors::{Publisher, SplitsUpdateMailbox, UploaderType}; let universe = Universe::with_accelerated_time(); let temp_dir = tempfile::tempdir().unwrap(); @@ -536,8 +532,7 @@ mod tests { // Create ParquetUploader let mock_metastore = MockMetastoreService::new(); let ram_storage = StdArc::new(RamStorage::default()); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let uploader = ParquetUploader::new( UploaderType::IndexUploader, quickwit_proto::metastore::MetastoreServiceClient::from_mock(mock_metastore), diff --git a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_e2e_test.rs similarity index 98% rename from quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_e2e_test.rs index d613fc96003..0c87ee902a3 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_e2e_test.rs @@ -15,7 +15,7 @@ //! End-to-end tests for the metrics pipeline. //! //! These tests wire up the full metrics pipeline: -//! ParquetDocProcessor → ParquetIndexer → ParquetUploader → ParquetPublisher +//! ParquetDocProcessor → ParquetIndexer → ParquetUploader → Publisher use std::sync::Arc; use std::sync::atomic::Ordering; @@ -37,8 +37,8 @@ use quickwit_proto::types::IndexUid; use quickwit_storage::RamStorage; use crate::actors::{ - ParquetDocProcessor, ParquetIndexer, ParquetPackager, ParquetPublisher, ParquetUploader, - PublisherType, SplitsUpdateMailbox, UploaderType, + ParquetDocProcessor, ParquetIndexer, ParquetPackager, ParquetUploader, Publisher, + SplitsUpdateMailbox, UploaderType, }; use crate::models::RawDocBatch; @@ -47,7 +47,7 @@ use crate::models::RawDocBatch; // ============================================================================= async fn wait_for_published_splits( - publisher_handle: &ActorHandle, + publisher_handle: &ActorHandle, expected_splits: u64, ) -> anyhow::Result<()> { wait_until_predicate( @@ -153,8 +153,9 @@ async fn test_metrics_pipeline_e2e() { quickwit_proto::metastore::MetastoreServiceClient::from_mock(mock_metastore); let ram_storage = Arc::new(RamStorage::default()); - let publisher = ParquetPublisher::new( - PublisherType::ParquetPublisher, + let publisher = Publisher::new( + super::METRICS_PUBLISHER_NAME, + quickwit_actors::QueueCapacity::Bounded(1), metastore_client.clone(), None, None, diff --git a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_indexer.rs similarity index 98% rename from quickwit/quickwit-indexing/src/actors/parquet_indexer.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_indexer.rs index 3d632e7cc76..5c85caf1e93 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_indexer.rs @@ -34,8 +34,9 @@ use tokio::runtime::Handle; use tracing::{debug, info, info_span, warn}; use ulid::Ulid; -use crate::actors::parquet_packager::{ParquetBatchForPackager, ParquetPackager}; -use crate::models::{NewPublishLock, NewPublishToken, ProcessedParquetBatch, PublishLock}; +use super::ProcessedParquetBatch; +use super::parquet_packager::{ParquetBatchForPackager, ParquetPackager}; +use crate::models::{NewPublishLock, NewPublishToken, PublishLock}; /// Default commit timeout for ParquetIndexer (60 seconds). // TODO: read from index config commit_timeout_secs. @@ -552,9 +553,8 @@ mod tests { use quickwit_storage::RamStorage; use super::*; - use crate::actors::{ - ParquetPackager, ParquetPublisher, ParquetUploader, SplitsUpdateMailbox, UploaderType, - }; + use crate::actors::metrics_pipeline::{ParquetPackager, ParquetUploader}; + use crate::actors::{Publisher, SplitsUpdateMailbox, UploaderType}; /// Create a test ParquetUploader and return its mailbox. fn create_test_uploader( @@ -562,8 +562,7 @@ mod tests { ) -> (Mailbox, ActorHandle) { let mock_metastore = MockMetastoreService::new(); let ram_storage = Arc::new(RamStorage::default()); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let uploader = ParquetUploader::new( UploaderType::IndexUploader, @@ -587,8 +586,7 @@ mod tests { .returning(|_| Ok(EmptyResponse {})); let ram_storage = Arc::new(RamStorage::default()); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let uploader = ParquetUploader::new( UploaderType::IndexUploader, @@ -916,8 +914,7 @@ mod tests { .returning(|_| Ok(EmptyResponse {})); let ram_storage = Arc::new(RamStorage::default()); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let uploader = ParquetUploader::new( UploaderType::IndexUploader, quickwit_proto::metastore::MetastoreServiceClient::from_mock(mock_metastore), diff --git a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_packager.rs similarity index 97% rename from quickwit/quickwit-indexing/src/actors/parquet_packager.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_packager.rs index b0950141b10..58390115b6e 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_packager.rs @@ -35,8 +35,8 @@ use serde::Serialize; use tokio::runtime::Handle; use tracing::{info, warn}; -use crate::actors::ParquetUploader; -use crate::actors::parquet_indexer::ParquetSplitBatch; +use super::ParquetUploader; +use super::parquet_indexer::ParquetSplitBatch; use crate::models::PublishLock; /// A concatenated RecordBatch ready to be written to a Parquet file. @@ -244,7 +244,7 @@ mod tests { use quickwit_storage::RamStorage; use super::*; - use crate::actors::{ParquetPublisher, SplitsUpdateMailbox, UploaderType}; + use crate::actors::{Publisher, SplitsUpdateMailbox, UploaderType}; fn create_test_uploader( universe: &Universe, @@ -255,8 +255,7 @@ mod tests { .returning(|_| Ok(EmptyResponse {})); let ram_storage = Arc::new(RamStorage::default()); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let uploader = ParquetUploader::new( UploaderType::IndexUploader, diff --git a/quickwit/quickwit-indexing/src/models/parquet_splits_update.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_splits_update.rs similarity index 100% rename from quickwit/quickwit-indexing/src/models/parquet_splits_update.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_splits_update.rs diff --git a/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_uploader.rs similarity index 97% rename from quickwit/quickwit-indexing/src/actors/parquet_uploader.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_uploader.rs index 2f5ea943749..e9fca543b8b 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/parquet_uploader.rs @@ -34,11 +34,11 @@ use quickwit_storage::Storage; use tokio::sync::{Semaphore, SemaphorePermit}; use tracing::{Instrument, Span, debug, info, instrument, warn}; +use super::{ParquetSplitBatch, ParquetSplitsUpdate}; use crate::actors::sequencer::SequencerCommand; use crate::actors::uploader::{SplitsUpdateMailbox, SplitsUpdateSender}; -use crate::actors::{ParquetPublisher, ParquetSplitBatch, UploaderCounters, UploaderType}; +use crate::actors::{Publisher, UploaderCounters, UploaderType}; use crate::metrics::INDEXER_METRICS; -use crate::models::ParquetSplitsUpdate; /// Concurrent upload permits for metrics uploader. /// Uses same permit pool as indexer uploads. @@ -53,7 +53,7 @@ pub struct ParquetUploader { uploader_type: UploaderType, metastore: MetastoreServiceClient, split_store: Arc, - split_update_mailbox: SplitsUpdateMailbox, + split_update_mailbox: SplitsUpdateMailbox, max_concurrent_uploads: usize, counters: UploaderCounters, } @@ -64,7 +64,7 @@ impl ParquetUploader { uploader_type: UploaderType, metastore: MetastoreServiceClient, split_store: Arc, - split_update_mailbox: SplitsUpdateMailbox, + split_update_mailbox: SplitsUpdateMailbox, max_concurrent_uploads: usize, ) -> Self { Self { @@ -405,8 +405,7 @@ mod tests { let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let mut mock_metastore = MockMetastoreService::new(); mock_metastore @@ -491,8 +490,7 @@ mod tests { let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let mut mock_metastore = MockMetastoreService::new(); mock_metastore @@ -576,8 +574,7 @@ mod tests { let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); - let (publisher_mailbox, _publisher_inbox) = - universe.create_test_mailbox::(); + let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox::(); let mut mock_metastore = MockMetastoreService::new(); // Should NOT call stage_metrics_splits for empty batch @@ -633,9 +630,8 @@ mod tests { let temp_dir = tempfile::tempdir().unwrap(); // Create a simple receiver actor to collect ParquetSplitsUpdate messages - // We use a test mailbox for ParquetPublisher to capture what would be sent - let (publisher_mailbox, publisher_inbox) = - universe.create_test_mailbox::(); + // We use a test mailbox for Publisher to capture what would be sent + let (publisher_mailbox, publisher_inbox) = universe.create_test_mailbox::(); // Create sequencer that forwards to publisher let sequencer = Sequencer::new(publisher_mailbox); diff --git a/quickwit/quickwit-indexing/src/actors/metrics_pipeline/pipeline.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/pipeline.rs new file mode 100644 index 00000000000..1388dab2441 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/pipeline.rs @@ -0,0 +1,509 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! MetricsPipeline actor — supervises the Parquet/DataFusion indexing pipeline. +//! +//! This is the metrics counterpart of `IndexingPipeline`. It spawns and +//! supervises the chain: +//! +//! ```text +//! Source → ParquetDocProcessor → ParquetIndexer → ParquetPackager → ParquetUploader → Publisher +//! ``` + +use std::collections::BTreeSet; +use std::sync::Arc; +use std::sync::atomic::Ordering; +use std::time::{Duration, Instant}; + +use async_trait::async_trait; +use quickwit_actors::{ + Actor, ActorContext, ActorExitStatus, ActorHandle, HEARTBEAT, Handler, Health, Mailbox, + QueueCapacity, Supervisable, +}; +use quickwit_common::KillSwitch; +use quickwit_common::metrics::OwnedGaugeGuard; +use quickwit_common::pubsub::EventBroker; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_config::{IndexingSettings, SourceConfig}; +use quickwit_ingest::IngesterPool; +use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::{MetastoreError, MetastoreServiceClient}; +use quickwit_proto::types::ShardId; +use quickwit_storage::{Storage, StorageResolver}; +use tracing::{debug, error, info, instrument}; + +use super::{ParquetDocProcessor, ParquetIndexer, ParquetPackager, ParquetUploader}; +use crate::actors::pipeline_shared::{ + SPAWN_PIPELINE_SEMAPHORE, SUPERVISE_INTERVAL, Spawn, SuperviseLoop, wait_duration_before_retry, +}; +use crate::actors::sequencer::Sequencer; +use crate::actors::{Publisher, SplitsUpdateMailbox, UploaderType}; +use crate::models::IndexingStatistics; +use crate::source::{ + AssignShards, Assignment, ProcessorMailbox, SourceActor, SourceRuntime, + quickwit_supported_sources, +}; + +struct MetricsPipelineHandles { + source_mailbox: Mailbox, + source_handle: ActorHandle, + doc_processor: ActorHandle, + indexer: ActorHandle, + packager: ActorHandle, + uploader: ActorHandle, + sequencer: ActorHandle>, + publisher: ActorHandle, + next_check_for_progress: Instant, +} + +impl MetricsPipelineHandles { + fn should_check_for_progress(&mut self) -> bool { + let now = Instant::now(); + let check_for_progress = now > self.next_check_for_progress; + if check_for_progress { + self.next_check_for_progress = now + *HEARTBEAT; + } + check_for_progress + } +} + +pub struct MetricsPipelineParams { + pub pipeline_id: IndexingPipelineId, + pub metastore: MetastoreServiceClient, + pub storage: Arc, + pub indexing_directory: TempDirectory, + pub indexing_settings: IndexingSettings, + pub max_concurrent_split_uploads: usize, + pub source_config: SourceConfig, + pub source_storage_resolver: StorageResolver, + pub ingester_pool: IngesterPool, + pub queues_dir_path: std::path::PathBuf, + pub params_fingerprint: u64, + pub event_broker: EventBroker, +} + +pub struct MetricsPipeline { + params: MetricsPipelineParams, + previous_generations_statistics: IndexingStatistics, + statistics: IndexingStatistics, + handles_opt: Option, + kill_switch: KillSwitch, + shard_ids: BTreeSet, + _indexing_pipelines_gauge_guard: OwnedGaugeGuard, +} + +#[async_trait] +impl Actor for MetricsPipeline { + type ObservableState = IndexingStatistics; + + fn observable_state(&self) -> Self::ObservableState { + self.statistics.clone() + } + + fn name(&self) -> String { + "MetricsPipeline".to_string() + } + + async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { + self.handle(Spawn::default(), ctx).await?; + self.handle(SuperviseLoop, ctx).await?; + Ok(()) + } + + async fn finalize( + &mut self, + _exit_status: &ActorExitStatus, + ctx: &ActorContext, + ) -> anyhow::Result<()> { + self.perform_observe(ctx); + Ok(()) + } +} + +impl MetricsPipeline { + pub fn new(params: MetricsPipelineParams) -> Self { + let indexing_pipelines_gauge = crate::metrics::INDEXER_METRICS + .indexing_pipelines + .with_label_values([¶ms.pipeline_id.index_uid.index_id]); + let indexing_pipelines_gauge_guard = OwnedGaugeGuard::from_gauge(indexing_pipelines_gauge); + let params_fingerprint = params.params_fingerprint; + MetricsPipeline { + params, + previous_generations_statistics: Default::default(), + handles_opt: None, + kill_switch: KillSwitch::default(), + statistics: IndexingStatistics { + params_fingerprint, + ..Default::default() + }, + shard_ids: Default::default(), + _indexing_pipelines_gauge_guard: indexing_pipelines_gauge_guard, + } + } + + fn supervisables(&self) -> Vec<&dyn Supervisable> { + match &self.handles_opt { + Some(handles) => { + vec![ + &handles.source_handle, + &handles.doc_processor, + &handles.indexer, + &handles.packager, + &handles.uploader, + &handles.sequencer, + &handles.publisher, + ] + } + None => Vec::new(), + } + } + + fn healthcheck(&self, check_for_progress: bool) -> Health { + let mut healthy_actors: Vec<&str> = Default::default(); + let mut failure_or_unhealthy_actors: Vec<&str> = Default::default(); + let mut success_actors: Vec<&str> = Default::default(); + for supervisable in self.supervisables() { + match supervisable.check_health(check_for_progress) { + Health::Healthy => { + healthy_actors.push(supervisable.name()); + } + Health::FailureOrUnhealthy => { + failure_or_unhealthy_actors.push(supervisable.name()); + } + Health::Success => { + success_actors.push(supervisable.name()); + } + } + } + + if !failure_or_unhealthy_actors.is_empty() { + error!( + pipeline_id=?self.params.pipeline_id, + generation=self.generation(), + healthy_actors=?healthy_actors, + failed_or_unhealthy_actors=?failure_or_unhealthy_actors, + success_actors=?success_actors, + "metrics pipeline failure" + ); + return Health::FailureOrUnhealthy; + } + if healthy_actors.is_empty() { + info!( + pipeline_id=?self.params.pipeline_id, + generation=self.generation(), + "metrics pipeline success" + ); + return Health::Success; + } + debug!( + pipeline_id=?self.params.pipeline_id, + generation=self.generation(), + healthy_actors=?healthy_actors, + failed_or_unhealthy_actors=?failure_or_unhealthy_actors, + success_actors=?success_actors, + "metrics pipeline running" + ); + Health::Healthy + } + + fn generation(&self) -> usize { + self.statistics.generation + } + + fn perform_observe(&mut self, ctx: &ActorContext) { + if let Some(handles) = &self.handles_opt { + handles.doc_processor.refresh_observe(); + handles.indexer.refresh_observe(); + handles.uploader.refresh_observe(); + handles.publisher.refresh_observe(); + + let doc_counters = handles.doc_processor.last_observation(); + let indexer_counters = handles.indexer.last_observation(); + let uploader_counters = handles.uploader.last_observation(); + let publisher_counters = handles.publisher.last_observation(); + + let mut stats = self.previous_generations_statistics.clone(); + stats.num_docs += doc_counters.valid_rows; + stats.num_invalid_docs += doc_counters.num_errors(); + stats.total_bytes_processed += doc_counters.bytes_total; + stats.num_local_splits += indexer_counters.batches_flushed; + stats.num_staged_splits += uploader_counters.num_staged_splits.load(Ordering::Relaxed); + stats.num_uploaded_splits += uploader_counters + .num_uploaded_splits + .load(Ordering::Relaxed); + stats.num_published_splits += publisher_counters.num_published_splits; + stats.num_empty_splits += publisher_counters.num_empty_splits; + stats.generation = self.statistics.generation; + stats.num_spawn_attempts = self.statistics.num_spawn_attempts; + self.statistics = stats; + } + self.statistics.params_fingerprint = self.params.params_fingerprint; + self.statistics.shard_ids.clone_from(&self.shard_ids); + ctx.observe(self); + } + + async fn perform_health_check( + &mut self, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + let check_for_progress = match &mut self.handles_opt { + Some(handles) => handles.should_check_for_progress(), + None => return Ok(()), + }; + let health = self.healthcheck(check_for_progress); + match health { + Health::Healthy => {} + Health::FailureOrUnhealthy => { + self.terminate().await; + let first_retry_delay = wait_duration_before_retry(0); + ctx.schedule_self_msg(first_retry_delay, Spawn { retry_count: 0 }); + } + Health::Success => { + return Err(ActorExitStatus::Success); + } + } + Ok(()) + } + + #[instrument( + name="spawn_metrics_pipeline", + level="info", + skip_all, + fields( + index=%self.params.pipeline_id.index_uid.index_id, + r#gen=self.generation() + ))] + async fn spawn_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { + let _spawn_pipeline_permit = ctx + .protect_future(SPAWN_PIPELINE_SEMAPHORE.acquire()) + .await + .expect("semaphore should not be closed"); + + self.statistics.num_spawn_attempts += 1; + self.kill_switch = ctx.kill_switch().child(); + + let index_id = &self.params.pipeline_id.index_uid.index_id; + let source_id = &self.params.pipeline_id.source_id; + + info!( + index_id, + source_id, + pipeline_uid=%self.params.pipeline_id.pipeline_uid, + root_dir=%self.params.indexing_directory.path().display(), + "spawning parquet indexing pipeline for metrics", + ); + + let (source_mailbox, source_inbox) = ctx + .spawn_ctx() + .create_mailbox::("SourceActor", QueueCapacity::Unbounded); + + // Publisher + let publisher = Publisher::new( + super::METRICS_PUBLISHER_NAME, + QueueCapacity::Bounded(1), + self.params.metastore.clone(), + None, + Some(source_mailbox.clone()), + ); + let (publisher_mailbox, publisher_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(publisher); + + // Sequencer + let sequencer = Sequencer::new(publisher_mailbox); + let (sequencer_mailbox, sequencer_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(sequencer); + + // ParquetUploader + let uploader = ParquetUploader::new( + UploaderType::IndexUploader, + self.params.metastore.clone(), + self.params.storage.clone(), + SplitsUpdateMailbox::Sequencer(sequencer_mailbox), + self.params.max_concurrent_split_uploads, + ); + let (uploader_mailbox, uploader_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(uploader); + + // ParquetPackager + let writer_config = quickwit_parquet_engine::storage::ParquetWriterConfig::default(); + let split_writer = quickwit_parquet_engine::storage::ParquetSplitWriter::new( + writer_config, + self.params.indexing_directory.path(), + ); + let packager = ParquetPackager::new(split_writer, uploader_mailbox); + let (packager_mailbox, packager_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(packager); + + // ParquetIndexer + let commit_timeout = + Duration::from_secs(self.params.indexing_settings.commit_timeout_secs as u64); + let indexer = ParquetIndexer::new( + self.params.pipeline_id.index_uid.clone(), + source_id.to_string(), + None, + packager_mailbox, + Some(commit_timeout), + ); + let (indexer_mailbox, indexer_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(indexer); + + // ParquetDocProcessor + let doc_processor = + ParquetDocProcessor::new(index_id.to_string(), source_id.to_string(), indexer_mailbox); + let (doc_processor_mailbox, doc_processor_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(doc_processor); + + // Source + let source_runtime = SourceRuntime { + pipeline_id: self.params.pipeline_id.clone(), + source_config: self.params.source_config.clone(), + metastore: self.params.metastore.clone(), + ingester_pool: self.params.ingester_pool.clone(), + queues_dir_path: self.params.queues_dir_path.clone(), + storage_resolver: self.params.source_storage_resolver.clone(), + event_broker: self.params.event_broker.clone(), + indexing_setting: self.params.indexing_settings.clone(), + }; + let source = ctx + .protect_future(quickwit_supported_sources().load_source(source_runtime)) + .await?; + let actor_source = SourceActor { + source, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), + }; + let (source_mailbox, source_handle) = ctx + .spawn_actor() + .set_mailboxes(source_mailbox, source_inbox) + .set_kill_switch(self.kill_switch.clone()) + .spawn(actor_source); + let assign_shards_message = AssignShards(Assignment { + shard_ids: self.shard_ids.clone(), + }); + source_mailbox.send_message(assign_shards_message).await?; + + self.previous_generations_statistics = self.statistics.clone(); + self.statistics.generation += 1; + self.handles_opt = Some(MetricsPipelineHandles { + source_mailbox, + source_handle, + doc_processor: doc_processor_handle, + indexer: indexer_handle, + packager: packager_handle, + uploader: uploader_handle, + sequencer: sequencer_handle, + publisher: publisher_handle, + next_check_for_progress: Instant::now() + *HEARTBEAT, + }); + Ok(()) + } + + async fn terminate(&mut self) { + self.kill_switch.kill(); + if let Some(handles) = self.handles_opt.take() { + tokio::join!( + handles.source_handle.kill(), + handles.indexer.kill(), + handles.packager.kill(), + handles.uploader.kill(), + handles.publisher.kill(), + ); + } + } +} + +#[async_trait] +impl Handler for MetricsPipeline { + type Reply = (); + async fn handle( + &mut self, + supervise_loop_token: SuperviseLoop, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + self.perform_observe(ctx); + self.perform_health_check(ctx).await?; + ctx.schedule_self_msg(SUPERVISE_INTERVAL, supervise_loop_token); + Ok(()) + } +} + +#[async_trait] +impl Handler for MetricsPipeline { + type Reply = (); + + async fn handle( + &mut self, + spawn: Spawn, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + if self.handles_opt.is_some() { + return Ok(()); + } + self.previous_generations_statistics.num_spawn_attempts = 1 + spawn.retry_count; + if let Err(spawn_error) = self.spawn_pipeline(ctx).await { + if let Some(MetastoreError::NotFound { .. }) = + spawn_error.downcast_ref::() + { + info!(error = ?spawn_error, "could not spawn metrics pipeline, index might have been deleted"); + return Err(ActorExitStatus::Success); + } + let retry_delay = wait_duration_before_retry(spawn.retry_count + 1); + error!(error = ?spawn_error, retry_count = spawn.retry_count, retry_delay = ?retry_delay, "error while spawning metrics pipeline, retrying after some time"); + ctx.schedule_self_msg( + retry_delay, + Spawn { + retry_count: spawn.retry_count + 1, + }, + ); + } + Ok(()) + } +} + +#[async_trait] +impl Handler for MetricsPipeline { + type Reply = (); + + async fn handle( + &mut self, + assign_shards_message: AssignShards, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + self.shard_ids + .clone_from(&assign_shards_message.0.shard_ids); + if let Some(handles) = &self.handles_opt { + info!( + shard_ids=?assign_shards_message.0.shard_ids, + "assigning shards to metrics pipeline" + ); + handles + .source_mailbox + .send_message(assign_shards_message) + .await?; + } + self.perform_observe(ctx); + Ok(()) + } +} diff --git a/quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/processed_parquet_batch.rs similarity index 100% rename from quickwit/quickwit-indexing/src/models/processed_parquet_batch.rs rename to quickwit/quickwit-indexing/src/actors/metrics_pipeline/processed_parquet_batch.rs diff --git a/quickwit/quickwit-indexing/src/actors/metrics_pipeline/publisher_impl.rs b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/publisher_impl.rs new file mode 100644 index 00000000000..56af0db9e52 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/metrics_pipeline/publisher_impl.rs @@ -0,0 +1,255 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! `Handler` implementation for `Publisher`, +//! specific to the metrics pipeline. + +use anyhow::Context; +use async_trait::async_trait; +use quickwit_actors::{ActorContext, ActorExitStatus, Handler}; +use quickwit_proto::metastore::{MetastoreService, PublishMetricsSplitsRequest}; +use tracing::{info, instrument}; + +use super::ParquetSplitsUpdate; +use crate::actors::publisher::{Publisher, serialize_checkpoint_delta, suggest_truncate}; + +pub(crate) const METRICS_PUBLISHER_NAME: &str = "ParquetPublisher"; + +#[async_trait] +impl Handler for Publisher { + type Reply = (); + + #[instrument(name = "parquet_publisher", parent = split_update.parent_span.id(), skip(self, ctx))] + async fn handle( + &mut self, + split_update: ParquetSplitsUpdate, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + let ParquetSplitsUpdate { + index_uid, + new_splits, + replaced_split_ids, + checkpoint_delta_opt, + publish_lock, + publish_token_opt, + .. + } = split_update; + + let index_checkpoint_delta_json_opt = serialize_checkpoint_delta(&checkpoint_delta_opt)?; + let split_ids: Vec = new_splits + .iter() + .map(|split| split.split_id.as_str().to_string()) + .collect(); + if let Some(_guard) = publish_lock.acquire().await { + let publish_request = PublishMetricsSplitsRequest { + index_uid: Some(index_uid.clone()), + staged_split_ids: split_ids.clone(), + replaced_split_ids: replaced_split_ids.clone(), + index_checkpoint_delta_json_opt, + publish_token_opt: publish_token_opt.clone(), + }; + ctx.protect_future(self.metastore.publish_metrics_splits(publish_request)) + .await + .context("failed to publish metrics splits")?; + } else { + info!( + split_ids=?split_ids, + "Splits' publish lock is dead." + ); + return Ok(()); + } + info!("publish-metrics-splits"); + suggest_truncate(ctx, &self.source_mailbox_opt, checkpoint_delta_opt).await; + + if !split_ids.is_empty() { + if replaced_split_ids.is_empty() { + self.counters.num_published_splits += 1; + } else { + self.counters.num_replace_operations += 1; + } + } else { + self.counters.num_empty_splits += 1; + } + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use quickwit_actors::{QueueCapacity, Universe}; + use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; + use quickwit_parquet_engine::split::{MetricsSplitMetadata, SplitId, TimeRange}; + use quickwit_proto::metastore::{EmptyResponse, MetastoreServiceClient, MockMetastoreService}; + use quickwit_proto::types::IndexUid; + use tracing::Span; + + use super::{METRICS_PUBLISHER_NAME, ParquetSplitsUpdate}; + use crate::actors::publisher::Publisher; + use crate::models::PublishLock; + + fn create_test_metrics_split_metadata(index_uid: &str, split_id: &str) -> MetricsSplitMetadata { + MetricsSplitMetadata::builder() + .index_uid(index_uid) + .split_id(SplitId::new(split_id)) + .time_range(TimeRange::new(1000, 2000)) + .num_rows(100) + .size_bytes(1024) + .build() + } + + #[tokio::test] + async fn test_metrics_publisher_publishes_splits() { + let universe = Universe::with_accelerated_time(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_publish_metrics_splits() + .withf(|request| { + request.index_uid().to_string().starts_with("test-index:") + && request.staged_split_ids == vec!["split-1".to_string()] + && request.replaced_split_ids.is_empty() + && request.index_checkpoint_delta_json_opt.is_some() + && request.publish_token_opt.is_none() + }) + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + let publisher = Publisher::new( + METRICS_PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + None, + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + let update = ParquetSplitsUpdate { + index_uid: IndexUid::for_test("test-index", 0), + new_splits: vec![create_test_metrics_split_metadata( + "test-index:00000000000000000000000000", + "split-1", + )], + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: Some(IndexCheckpointDelta { + source_id: "test-source".to_string(), + source_delta: SourceCheckpointDelta::from_range(0..10), + }), + publish_lock: PublishLock::default(), + publish_token_opt: None, + parent_span: Span::none(), + }; + + publisher_mailbox.send_message(update).await.unwrap(); + + let observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(observation.num_published_splits, 1); + assert_eq!(observation.num_replace_operations, 0); + assert_eq!(observation.num_empty_splits, 0); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_metrics_publisher_handles_empty_splits() { + let universe = Universe::with_accelerated_time(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_publish_metrics_splits() + .withf(|request| { + request.index_uid().to_string().starts_with("test-index:") + && request.staged_split_ids.is_empty() + && request.replaced_split_ids.is_empty() + && request.index_checkpoint_delta_json_opt.is_some() + }) + .times(1) + .returning(|_| Ok(EmptyResponse {})); + + let publisher = Publisher::new( + METRICS_PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + None, + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + let update = ParquetSplitsUpdate { + index_uid: IndexUid::for_test("test-index", 0), + new_splits: Vec::new(), + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: Some(IndexCheckpointDelta { + source_id: "test-source".to_string(), + source_delta: SourceCheckpointDelta::from_range(0..1), + }), + publish_lock: PublishLock::default(), + publish_token_opt: None, + parent_span: Span::none(), + }; + + publisher_mailbox.send_message(update).await.unwrap(); + + let observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(observation.num_published_splits, 0); + assert_eq!(observation.num_replace_operations, 0); + assert_eq!(observation.num_empty_splits, 1); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_metrics_publisher_respects_publish_lock() { + let universe = Universe::with_accelerated_time(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore.expect_publish_metrics_splits().never(); + + let publisher = Publisher::new( + METRICS_PUBLISHER_NAME, + QueueCapacity::Bounded(1), + MetastoreServiceClient::from_mock(mock_metastore), + None, + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + let publish_lock = PublishLock::default(); + publish_lock.kill().await; + + let update = ParquetSplitsUpdate { + index_uid: IndexUid::for_test("test-index", 0), + new_splits: vec![create_test_metrics_split_metadata( + "test-index:00000000000000000000000000", + "split-1", + )], + replaced_split_ids: Vec::new(), + checkpoint_delta_opt: Some(IndexCheckpointDelta { + source_id: "test-source".to_string(), + source_delta: SourceCheckpointDelta::from_range(0..10), + }), + publish_lock, + publish_token_opt: None, + parent_span: Span::none(), + }; + + publisher_mailbox.send_message(update).await.unwrap(); + + let observation = publisher_handle.process_pending_and_observe().await.state; + assert_eq!(observation.num_published_splits, 0); + assert_eq!(observation.num_replace_operations, 0); + assert_eq!(observation.num_empty_splits, 0); + + universe.assert_quit().await; + } +} diff --git a/quickwit/quickwit-indexing/src/actors/mod.rs b/quickwit/quickwit-indexing/src/actors/mod.rs index 9806d4e03cf..3d56a2cab5a 100644 --- a/quickwit/quickwit-indexing/src/actors/mod.rs +++ b/quickwit/quickwit-indexing/src/actors/mod.rs @@ -18,60 +18,43 @@ mod index_serializer; mod indexer; mod indexing_pipeline; mod indexing_service; +mod log_publisher_impl; mod merge_executor; -mod merge_pipeline; +pub(crate) mod merge_pipeline; mod merge_planner; mod merge_scheduler_service; mod merge_split_downloader; +#[cfg(feature = "metrics")] +pub(crate) mod metrics_pipeline; mod packager; -mod parquet_doc_processor; -mod parquet_indexer; -mod parquet_packager; -mod parquet_uploader; +pub(crate) mod pipeline_shared; mod publisher; mod sequencer; mod uploader; #[cfg(feature = "vrl")] mod vrl_processing; -#[cfg(test)] -#[allow( - clippy::disallowed_methods, - clippy::needless_borrow, - clippy::unnecessary_map_or -)] -mod parquet_e2e_test; - pub use doc_processor::{DocProcessor, DocProcessorCounters}; pub use index_serializer::IndexSerializer; pub use indexer::{Indexer, IndexerCounters}; pub use indexing_pipeline::{IndexingPipeline, IndexingPipelineParams}; -pub use indexing_service::{INDEXING_DIR_NAME, IndexingService, IndexingServiceCounters}; +pub use indexing_service::{ + BoxPipelineHandle, INDEXING_DIR_NAME, IndexingService, IndexingServiceCounters, +}; +pub(crate) use log_publisher_impl::{MERGE_PUBLISHER_NAME, PUBLISHER_NAME}; pub use merge_executor::{MergeExecutor, combine_partition_ids, merge_split_attrs}; -pub use merge_pipeline::{FinishPendingMergesAndShutdownPipeline, MergePipeline}; -pub(crate) use merge_planner::{MergePlanner, RunFinalizeMergePolicyAndQuit}; +pub use merge_pipeline::{ + FinishPendingMergesAndShutdownPipeline, MergePipeline, MergePipelineParams, +}; +pub(crate) use merge_planner::MergePlanner; +#[cfg(test)] +pub(crate) use merge_planner::RunFinalizeMergePolicyAndQuit; pub use merge_scheduler_service::{MergePermit, MergeSchedulerService, schedule_merge}; pub use merge_split_downloader::MergeSplitDownloader; +#[cfg(feature = "metrics")] +pub use metrics_pipeline::*; pub use packager::Packager; -pub use parquet_doc_processor::{ - ParquetDocProcessor, ParquetDocProcessorCounters, ParquetDocProcessorError, is_arrow_ipc, -}; -pub use parquet_indexer::{ParquetIndexer, ParquetIndexerCounters, ParquetSplitBatch}; -pub use parquet_packager::{ParquetBatchForPackager, ParquetPackager, ParquetPackagerCounters}; -pub use parquet_uploader::ParquetUploader; -pub use publisher::{ParquetPublisher, Publisher, PublisherCounters, PublisherType}; -use quickwit_actors::{Actor, Handler}; +pub use publisher::{Publisher, PublisherCounters}; pub use quickwit_proto::indexing::IndexingError; pub use sequencer::Sequencer; pub use uploader::{SplitsUpdateMailbox, Uploader, UploaderCounters, UploaderType}; - -use crate::models::{NewPublishLock, NewPublishToken, RawDocBatch}; - -/// Trait alias for actor types that can process document batches. -pub trait Processor: - Actor + Handler + Handler + Handler -{ -} - -impl Processor for T where T: Actor + Handler + Handler + Handler -{} diff --git a/quickwit/quickwit-indexing/src/actors/pipeline_shared.rs b/quickwit/quickwit-indexing/src/actors/pipeline_shared.rs new file mode 100644 index 00000000000..bc1939a2089 --- /dev/null +++ b/quickwit/quickwit-indexing/src/actors/pipeline_shared.rs @@ -0,0 +1,136 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Shared infrastructure for indexing pipeline supervisors (logs and metrics). + +use std::time::Duration; + +use tokio::sync::Semaphore; + +pub(crate) const SUPERVISE_INTERVAL: Duration = Duration::from_secs(1); + +const MAX_RETRY_DELAY: Duration = Duration::from_secs(600); // 10 min. + +#[derive(Debug)] +pub(crate) struct SuperviseLoop; + +/// Calculates the wait time based on retry count. +// retry_count, wait_time +// 0 1s +// 1 2s +// 2 4s +// 3 8s +// ... +// >=8 5mn +pub(crate) fn wait_duration_before_retry(retry_count: usize) -> Duration { + // Protect against a `retry_count` that will lead to an overflow. + let max_power = (retry_count as u32).min(31); + Duration::from_secs(2u64.pow(max_power)).min(MAX_RETRY_DELAY) +} + +/// Spawning an indexing pipeline puts a lot of pressure on the file system, metastore, etc. so +/// we rely on this semaphore to limit the number of indexing pipelines that can be spawned +/// concurrently. +/// See also . +pub(crate) static SPAWN_PIPELINE_SEMAPHORE: Semaphore = Semaphore::const_new(10); + +#[derive(Clone, Copy, Debug, Default)] +pub(crate) struct Spawn { + pub(crate) retry_count: usize, +} + +// --------------------------------------------------------------------------- +// Pipeline trait — type-erased handle for any indexing pipeline actor +// --------------------------------------------------------------------------- + +use async_trait::async_trait; +use quickwit_actors::{ + Actor, ActorExitStatus, ActorHandle, ActorState, DeferableReplyHandler, Health, Mailbox, + Observation, SendError, Supervisable, +}; +use quickwit_proto::indexing::IndexingPipelineId; + +use crate::models::IndexingStatistics; +use crate::source::AssignShards; + +/// Trait that abstracts over the concrete pipeline actor type +/// (`IndexingPipeline` or `MetricsPipeline`). This allows `PipelineHandle` +/// to hold a single `Box`. +#[async_trait] +pub trait PipelineHandle: Send + Sync { + fn indexing_pipeline_id(&self) -> &IndexingPipelineId; + fn state(&self) -> ActorState; + fn refresh_observe(&self); + fn last_observation(&self) -> IndexingStatistics; + fn check_health(&self, check_for_progress: bool) -> Health; + async fn send_assign_shards(&self, message: AssignShards) -> Result<(), SendError>; + async fn observe(&self) -> Observation; + async fn join(self: Box) -> (ActorExitStatus, IndexingStatistics); + async fn quit(self: Box) -> (ActorExitStatus, IndexingStatistics); + async fn kill(self: Box); +} + +/// Generic wrapper that implements `PipelineHandle` for any actor with the right +/// observable state and message handlers. +pub(crate) struct ActorPipeline> { + pub pipeline_id: IndexingPipelineId, + pub mailbox: Mailbox, + pub handle: ActorHandle, +} + +#[async_trait] +impl PipelineHandle for ActorPipeline +where A: Actor + DeferableReplyHandler +{ + fn indexing_pipeline_id(&self) -> &IndexingPipelineId { + &self.pipeline_id + } + + fn state(&self) -> ActorState { + self.handle.state() + } + + fn refresh_observe(&self) { + self.handle.refresh_observe(); + } + + fn last_observation(&self) -> IndexingStatistics { + self.handle.last_observation().clone() + } + + fn check_health(&self, check_for_progress: bool) -> Health { + self.handle.check_health(check_for_progress) + } + + async fn send_assign_shards(&self, message: AssignShards) -> Result<(), SendError> { + self.mailbox.send_message(message).await?; + Ok(()) + } + + async fn observe(&self) -> Observation { + self.handle.observe().await + } + + async fn join(self: Box) -> (ActorExitStatus, IndexingStatistics) { + self.handle.join().await + } + + async fn quit(self: Box) -> (ActorExitStatus, IndexingStatistics) { + self.handle.quit().await + } + + async fn kill(self: Box) { + let _ = self.handle.kill().await; + } +} diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index c4ec75b1a8a..98716fa127b 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -14,22 +14,14 @@ use anyhow::Context; use async_trait::async_trait; -use fail::fail_point; -use quickwit_actors::{Actor, ActorContext, Handler, Mailbox, QueueCapacity}; +use quickwit_actors::{Actor, ActorContext, Mailbox, QueueCapacity}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; -use quickwit_proto::metastore::{ - MetastoreService, MetastoreServiceClient, PublishMetricsSplitsRequest, PublishSplitsRequest, -}; +use quickwit_proto::metastore::MetastoreServiceClient; use serde::Serialize; -use tracing::{info, instrument}; -use crate::actors::{DocProcessor, MergePlanner, ParquetDocProcessor, Processor}; -use crate::models::{NewSplits, ParquetSplitsUpdate, SplitsUpdate}; +use crate::actors::MergePlanner; use crate::source::{SourceActor, SuggestTruncate}; -/// Type alias for the metrics publisher specialization. -pub type ParquetPublisher = Publisher; - #[derive(Clone, Debug, Default, Serialize)] pub struct PublisherCounters { pub num_published_splits: u64, @@ -37,58 +29,32 @@ pub struct PublisherCounters { pub num_empty_splits: u64, } -#[derive(Clone, Copy, Debug)] -pub enum PublisherType { - MainPublisher, - MergePublisher, - ParquetPublisher, -} - -impl PublisherType { - pub fn actor_name(&self) -> &'static str { - match self { - PublisherType::MainPublisher => "Publisher", - PublisherType::MergePublisher => "MergePublisher", - PublisherType::ParquetPublisher => "ParquetPublisher", - } - } -} - /// Disconnect the merge planner loop back. /// This message is used to cut the merge pipeline loop, and let it terminate. #[derive(Debug)] pub(crate) struct DisconnectMergePlanner; -pub struct Publisher { - publisher_type: PublisherType, - metastore: MetastoreServiceClient, - merge_planner_mailbox_opt: Option>, - source_mailbox_opt: Option>>, - counters: PublisherCounters, +#[derive(Clone)] +pub struct Publisher { + pub(crate) name: &'static str, + pub(crate) queue_capacity: QueueCapacity, + pub(crate) metastore: MetastoreServiceClient, + pub(crate) merge_planner_mailbox_opt: Option>, + pub(crate) source_mailbox_opt: Option>, + pub(crate) counters: PublisherCounters, } -// Manual Clone impl to avoid a spurious `P: Clone` bound from #[derive(Clone)]. -impl Clone for Publisher

{ - fn clone(&self) -> Self { - Self { - publisher_type: self.publisher_type, - metastore: self.metastore.clone(), - merge_planner_mailbox_opt: self.merge_planner_mailbox_opt.clone(), - source_mailbox_opt: self.source_mailbox_opt.clone(), - counters: self.counters.clone(), - } - } -} - -impl Publisher

{ +impl Publisher { pub fn new( - publisher_type: PublisherType, + name: &'static str, + queue_capacity: QueueCapacity, metastore: MetastoreServiceClient, merge_planner_mailbox_opt: Option>, - source_mailbox_opt: Option>>, - ) -> Publisher

{ + source_mailbox_opt: Option>, + ) -> Publisher { Publisher { - publisher_type, + name, + queue_capacity, metastore, merge_planner_mailbox_opt, source_mailbox_opt, @@ -97,7 +63,7 @@ impl Publisher

{ } } -fn serialize_checkpoint_delta( +pub(crate) fn serialize_checkpoint_delta( checkpoint_delta_opt: &Option, ) -> anyhow::Result> { checkpoint_delta_opt @@ -107,9 +73,9 @@ fn serialize_checkpoint_delta( .context("failed to serialize `IndexCheckpointDelta`") } -async fn suggest_truncate( - ctx: &ActorContext, - source_mailbox_opt: &Option>>, +pub(crate) async fn suggest_truncate( + ctx: &ActorContext, + source_mailbox_opt: &Option>, checkpoint_delta_opt: Option, ) { if let Some(source_mailbox) = source_mailbox_opt.as_ref() @@ -125,7 +91,7 @@ async fn suggest_truncate( } #[async_trait] -impl Actor for Publisher

{ +impl Actor for Publisher { type ObservableState = PublisherCounters; fn observable_state(&self) -> Self::ObservableState { @@ -133,590 +99,10 @@ impl Actor for Publisher

{ } fn name(&self) -> String { - self.publisher_type.actor_name().to_string() + self.name.to_string() } fn queue_capacity(&self) -> QueueCapacity { - match self.publisher_type { - PublisherType::MainPublisher => QueueCapacity::Bounded(1), - PublisherType::MergePublisher => QueueCapacity::Unbounded, - PublisherType::ParquetPublisher => QueueCapacity::Bounded(1), - } - } -} - -#[async_trait] -impl Handler for Publisher

{ - type Reply = (); - - async fn handle( - &mut self, - _: DisconnectMergePlanner, - _ctx: &ActorContext, - ) -> Result<(), quickwit_actors::ActorExitStatus> { - info!("disconnecting merge planner mailbox"); - self.merge_planner_mailbox_opt = None; - Ok(()) - } -} - -#[async_trait] -impl Handler for Publisher { - type Reply = (); - - #[instrument(name="publisher", parent=split_update.parent_span.id(), skip(self, ctx))] - async fn handle( - &mut self, - split_update: SplitsUpdate, - ctx: &ActorContext, - ) -> Result<(), quickwit_actors::ActorExitStatus> { - fail_point!("publisher:before"); - - let SplitsUpdate { - index_uid, - new_splits, - replaced_split_ids, - checkpoint_delta_opt, - publish_lock, - publish_token_opt, - .. - } = split_update; - - let index_checkpoint_delta_json_opt = serialize_checkpoint_delta(&checkpoint_delta_opt)?; - let split_ids: Vec = new_splits - .iter() - .map(|split| split.split_id.clone()) - .collect(); - if let Some(_guard) = publish_lock.acquire().await { - let publish_splits_request = PublishSplitsRequest { - index_uid: Some(index_uid), - staged_split_ids: split_ids.clone(), - replaced_split_ids: replaced_split_ids.clone(), - index_checkpoint_delta_json_opt, - publish_token_opt: publish_token_opt.clone(), - }; - ctx.protect_future(self.metastore.publish_splits(publish_splits_request)) - .await - .context("failed to publish splits")?; - } else { - // TODO: Remove the junk right away? - info!( - split_ids=?split_ids, - "Splits' publish lock is dead." - ); - return Ok(()); - } - info!("publish-new-splits"); - suggest_truncate(ctx, &self.source_mailbox_opt, checkpoint_delta_opt).await; - - if !new_splits.is_empty() { - // The merge planner is not necessarily awake and this is not an error. - // For instance, when a source reaches its end, and the last "new" split - // has been packaged, the packager finalizer sends a message to the merge - // planner in order to stop it. - if let Some(merge_planner_mailbox) = self.merge_planner_mailbox_opt.as_ref() { - let _ = ctx - .send_message(merge_planner_mailbox, NewSplits { new_splits }) - .await; - } - - if replaced_split_ids.is_empty() { - self.counters.num_published_splits += 1; - } else { - self.counters.num_replace_operations += 1; - } - } else { - self.counters.num_empty_splits += 1; - } - fail_point!("publisher:after"); - Ok(()) - } -} - -#[async_trait] -impl Handler for Publisher { - type Reply = (); - - #[instrument(name = "parquet_publisher", parent = split_update.parent_span.id(), skip(self, ctx))] - async fn handle( - &mut self, - split_update: ParquetSplitsUpdate, - ctx: &ActorContext, - ) -> Result<(), quickwit_actors::ActorExitStatus> { - let ParquetSplitsUpdate { - index_uid, - new_splits, - replaced_split_ids, - checkpoint_delta_opt, - publish_lock, - publish_token_opt, - .. - } = split_update; - - let index_checkpoint_delta_json_opt = serialize_checkpoint_delta(&checkpoint_delta_opt)?; - let split_ids: Vec = new_splits - .iter() - .map(|split| split.split_id.as_str().to_string()) - .collect(); - if let Some(_guard) = publish_lock.acquire().await { - let publish_request = PublishMetricsSplitsRequest { - index_uid: Some(index_uid.clone()), - staged_split_ids: split_ids.clone(), - replaced_split_ids: replaced_split_ids.clone(), - index_checkpoint_delta_json_opt, - publish_token_opt: publish_token_opt.clone(), - }; - ctx.protect_future(self.metastore.publish_metrics_splits(publish_request)) - .await - .context("failed to publish metrics splits")?; - } else { - info!( - split_ids=?split_ids, - "Splits' publish lock is dead." - ); - return Ok(()); - } - info!("publish-metrics-splits"); - suggest_truncate(ctx, &self.source_mailbox_opt, checkpoint_delta_opt).await; - - if !split_ids.is_empty() { - if replaced_split_ids.is_empty() { - self.counters.num_published_splits += 1; - } else { - self.counters.num_replace_operations += 1; - } - } else { - self.counters.num_empty_splits += 1; - } - Ok(()) - } -} - -#[cfg(test)] -mod tests { - use quickwit_actors::Universe; - use quickwit_metastore::checkpoint::{ - IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, - }; - use quickwit_metastore::{PublishSplitsRequestExt, SplitMetadata}; - use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; - use quickwit_proto::types::{IndexUid, Position}; - use tracing::Span; - - use super::*; - use crate::models::PublishLock; - use crate::source::SuggestTruncate; - - #[tokio::test] - async fn test_publisher_publish_operation() { - let universe = Universe::with_accelerated_time(); - let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); - let mut mock_metastore = MockMetastoreService::new(); - let ref_index_uid_clone = ref_index_uid.clone(); - mock_metastore - .expect_publish_splits() - .withf(move |publish_splits_request| { - let checkpoint_delta: IndexCheckpointDelta = publish_splits_request - .deserialize_index_checkpoint() - .unwrap() - .unwrap(); - publish_splits_request.index_uid() == &ref_index_uid_clone - && checkpoint_delta.source_id == "source" - && publish_splits_request.staged_split_ids[..] == ["split"] - && publish_splits_request.replaced_split_ids.is_empty() - && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) - }) - .times(1) - .returning(|_| Ok(EmptyResponse {})); - let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); - - let (source_mailbox, source_inbox) = universe.create_test_mailbox(); - - let publisher = Publisher::new( - PublisherType::MainPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - Some(merge_planner_mailbox), - Some(source_mailbox), - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - assert!( - publisher_mailbox - .send_message(SplitsUpdate { - index_uid: ref_index_uid.clone(), - new_splits: vec![SplitMetadata { - split_id: "split".to_string(), - ..Default::default() - }], - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: Some(IndexCheckpointDelta { - source_id: "source".to_string(), - source_delta: SourceCheckpointDelta::from_range(1..3), - }), - publish_lock: PublishLock::default(), - publish_token_opt: None, - merge_task: None, - parent_span: tracing::Span::none(), - }) - .await - .is_ok() - ); - - let publisher_observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(publisher_observation.num_published_splits, 1); - - let suggest_truncate_checkpoints: Vec = source_inbox - .drain_for_test_typed::() - .into_iter() - .map(|msg| msg.0) - .collect(); - - assert_eq!(suggest_truncate_checkpoints.len(), 1); - assert_eq!( - suggest_truncate_checkpoints[0] - .position_for_partition(&PartitionId::default()) - .unwrap(), - &Position::offset(2u64) - ); - - let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); - assert_eq!(merger_msgs.len(), 1); - assert_eq!(merger_msgs[0].new_splits.len(), 1); - universe.assert_quit().await; - } - - #[tokio::test] - async fn test_publisher_publish_operation_with_empty_splits() { - let universe = Universe::with_accelerated_time(); - let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); - let mut mock_metastore = MockMetastoreService::new(); - let ref_index_uid_clone = ref_index_uid.clone(); - mock_metastore - .expect_publish_splits() - .withf(move |publish_splits_request| { - let checkpoint_delta: IndexCheckpointDelta = publish_splits_request - .deserialize_index_checkpoint() - .unwrap() - .unwrap(); - publish_splits_request.index_uid() == &ref_index_uid_clone - && checkpoint_delta.source_id == "source" - && publish_splits_request.staged_split_ids.is_empty() - && publish_splits_request.replaced_split_ids.is_empty() - && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) - }) - .times(1) - .returning(|_| Ok(EmptyResponse {})); - let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); - - let (source_mailbox, source_inbox) = universe.create_test_mailbox(); - - let publisher = Publisher::new( - PublisherType::MainPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - Some(merge_planner_mailbox), - Some(source_mailbox), - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - assert!( - publisher_mailbox - .send_message(SplitsUpdate { - index_uid: ref_index_uid.clone(), - new_splits: Vec::new(), - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: Some(IndexCheckpointDelta { - source_id: "source".to_string(), - source_delta: SourceCheckpointDelta::from_range(1..3), - }), - publish_lock: PublishLock::default(), - publish_token_opt: None, - merge_task: None, - parent_span: tracing::Span::none(), - }) - .await - .is_ok() - ); - - let publisher_observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(publisher_observation.num_published_splits, 0); - assert_eq!(publisher_observation.num_replace_operations, 0); - assert_eq!(publisher_observation.num_empty_splits, 1); - - let suggest_truncate_checkpoints: Vec = source_inbox - .drain_for_test_typed::() - .into_iter() - .map(|msg| msg.0) - .collect(); - - assert_eq!(suggest_truncate_checkpoints.len(), 1); - assert_eq!( - suggest_truncate_checkpoints[0] - .position_for_partition(&PartitionId::default()) - .unwrap(), - &Position::offset(2u64) - ); - - let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); - assert_eq!(merger_msgs.len(), 0); - universe.assert_quit().await; - } - - #[tokio::test] - async fn test_publisher_replace_operation() { - let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastoreService::new(); - let ref_index_uid: IndexUid = IndexUid::for_test("index", 1); - let ref_index_uid_clone = ref_index_uid.clone(); - mock_metastore - .expect_publish_splits() - .withf(move |publish_splits_requests| { - publish_splits_requests.index_uid() == &ref_index_uid_clone - && publish_splits_requests.staged_split_ids[..] == ["split3"] - && publish_splits_requests.replaced_split_ids[..] == ["split1", "split2"] - && publish_splits_requests - .index_checkpoint_delta_json_opt() - .is_empty() - }) - .times(1) - .returning(|_| Ok(EmptyResponse {})); - let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); - let publisher = Publisher::new( - PublisherType::MainPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - Some(merge_planner_mailbox), - None, - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - let publisher_message = SplitsUpdate { - index_uid: ref_index_uid.clone(), - new_splits: vec![SplitMetadata { - split_id: "split3".to_string(), - ..Default::default() - }], - replaced_split_ids: vec!["split1".to_string(), "split2".to_string()], - checkpoint_delta_opt: None, - publish_lock: PublishLock::default(), - publish_token_opt: None, - merge_task: None, - parent_span: Span::none(), - }; - assert!( - publisher_mailbox - .send_message(publisher_message) - .await - .is_ok() - ); - let publisher_observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(publisher_observation.num_published_splits, 0); - assert_eq!(publisher_observation.num_replace_operations, 1); - let merge_planner_msgs = merge_planner_inbox.drain_for_test_typed::(); - assert_eq!(merge_planner_msgs.len(), 1); - assert_eq!(merge_planner_msgs[0].new_splits.len(), 1); - universe.assert_quit().await; - } - - #[tokio::test] - async fn publisher_acquires_publish_lock() { - let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastoreService::new(); - mock_metastore.expect_publish_splits().never(); - let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); - - let publisher = Publisher::new( - PublisherType::MainPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - Some(merge_planner_mailbox), - None, - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - let publish_lock = PublishLock::default(); - publish_lock.kill().await; - - publisher_mailbox - .send_message(SplitsUpdate { - index_uid: IndexUid::new_with_random_ulid("index"), - new_splits: vec![SplitMetadata::for_test("test-split".to_string())], - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: None, - publish_lock, - publish_token_opt: None, - merge_task: None, - parent_span: Span::none(), - }) - .await - .unwrap(); - - let publisher_observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(publisher_observation.num_published_splits, 0); - - let merger_messages = merge_planner_inbox.drain_for_test(); - assert!(merger_messages.is_empty()); - universe.assert_quit().await; - } -} - -#[cfg(test)] -mod parquet_publisher_tests { - use quickwit_actors::Universe; - use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_parquet_engine::split::{MetricsSplitMetadata, SplitId, TimeRange}; - use quickwit_proto::metastore::{EmptyResponse, MetastoreServiceClient, MockMetastoreService}; - use quickwit_proto::types::IndexUid; - use tracing::Span; - - use super::*; - use crate::models::PublishLock; - - fn create_test_metrics_split_metadata(index_uid: &str, split_id: &str) -> MetricsSplitMetadata { - MetricsSplitMetadata::builder() - .index_uid(index_uid) - .split_id(SplitId::new(split_id)) - .time_range(TimeRange::new(1000, 2000)) - .num_rows(100) - .size_bytes(1024) - .build() - } - - #[tokio::test] - async fn test_metrics_publisher_publishes_splits() { - let universe = Universe::with_accelerated_time(); - - let mut mock_metastore = MockMetastoreService::new(); - mock_metastore - .expect_publish_metrics_splits() - .withf(|request| { - request.index_uid().to_string().starts_with("test-index:") - && request.staged_split_ids == vec!["split-1".to_string()] - && request.replaced_split_ids.is_empty() - && request.index_checkpoint_delta_json_opt.is_some() - && request.publish_token_opt.is_none() - }) - .times(1) - .returning(|_| Ok(EmptyResponse {})); - - let publisher = ParquetPublisher::new( - PublisherType::ParquetPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - None, - None, - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - let update = ParquetSplitsUpdate { - index_uid: IndexUid::for_test("test-index", 0), - new_splits: vec![create_test_metrics_split_metadata( - "test-index:00000000000000000000000000", - "split-1", - )], - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: Some(IndexCheckpointDelta { - source_id: "test-source".to_string(), - source_delta: SourceCheckpointDelta::from_range(0..10), - }), - publish_lock: PublishLock::default(), - publish_token_opt: None, - parent_span: Span::none(), - }; - - publisher_mailbox.send_message(update).await.unwrap(); - - let observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(observation.num_published_splits, 1); - assert_eq!(observation.num_replace_operations, 0); - assert_eq!(observation.num_empty_splits, 0); - - universe.assert_quit().await; - } - - #[tokio::test] - async fn test_metrics_publisher_handles_empty_splits() { - let universe = Universe::with_accelerated_time(); - - let mut mock_metastore = MockMetastoreService::new(); - mock_metastore - .expect_publish_metrics_splits() - .withf(|request| { - request.index_uid().to_string().starts_with("test-index:") - && request.staged_split_ids.is_empty() - && request.replaced_split_ids.is_empty() - && request.index_checkpoint_delta_json_opt.is_some() - }) - .times(1) - .returning(|_| Ok(EmptyResponse {})); - - let publisher = ParquetPublisher::new( - PublisherType::ParquetPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - None, - None, - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - let update = ParquetSplitsUpdate { - index_uid: IndexUid::for_test("test-index", 0), - new_splits: Vec::new(), - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: Some(IndexCheckpointDelta { - source_id: "test-source".to_string(), - source_delta: SourceCheckpointDelta::from_range(0..1), - }), - publish_lock: PublishLock::default(), - publish_token_opt: None, - parent_span: Span::none(), - }; - - publisher_mailbox.send_message(update).await.unwrap(); - - let observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(observation.num_published_splits, 0); - assert_eq!(observation.num_replace_operations, 0); - assert_eq!(observation.num_empty_splits, 1); - - universe.assert_quit().await; - } - - #[tokio::test] - async fn test_metrics_publisher_respects_publish_lock() { - let universe = Universe::with_accelerated_time(); - - let mut mock_metastore = MockMetastoreService::new(); - mock_metastore.expect_publish_metrics_splits().never(); - - let publisher = ParquetPublisher::new( - PublisherType::ParquetPublisher, - MetastoreServiceClient::from_mock(mock_metastore), - None, - None, - ); - let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); - - let publish_lock = PublishLock::default(); - publish_lock.kill().await; - - let update = ParquetSplitsUpdate { - index_uid: IndexUid::for_test("test-index", 0), - new_splits: vec![create_test_metrics_split_metadata( - "test-index:00000000000000000000000000", - "split-1", - )], - replaced_split_ids: Vec::new(), - checkpoint_delta_opt: Some(IndexCheckpointDelta { - source_id: "test-source".to_string(), - source_delta: SourceCheckpointDelta::from_range(0..10), - }), - publish_lock, - publish_token_opt: None, - parent_span: Span::none(), - }; - - publisher_mailbox.send_message(update).await.unwrap(); - - let observation = publisher_handle.process_pending_and_observe().await.state; - assert_eq!(observation.num_published_splits, 0); - assert_eq!(observation.num_replace_operations, 0); - assert_eq!(observation.num_empty_splits, 0); - - universe.assert_quit().await; + self.queue_capacity.clone() } } diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index 448b7a4e312..c20fddad1a6 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -78,8 +78,7 @@ pub enum UploaderType { /// publishing order does not matter. In this case, we can just send the update directly to the /// publisher. /// -/// The default type parameter `P = Publisher` preserves backward compatibility with the -/// standard logs pipeline. For the metrics pipeline, use `SplitsUpdateMailbox`. +/// The default type parameter `P = Publisher` is used by both the logs and metrics pipelines. pub enum SplitsUpdateMailbox { Sequencer(Mailbox>), Publisher(Mailbox

), diff --git a/quickwit/quickwit-indexing/src/lib.rs b/quickwit/quickwit-indexing/src/lib.rs index 2c2b28a09d7..516a7b22ae4 100644 --- a/quickwit/quickwit-indexing/src/lib.rs +++ b/quickwit/quickwit-indexing/src/lib.rs @@ -26,8 +26,8 @@ use tracing::info; use crate::actors::MergeSchedulerService; pub use crate::actors::{ - FinishPendingMergesAndShutdownPipeline, IndexingError, IndexingPipeline, - IndexingPipelineParams, IndexingService, PublisherType, Sequencer, SplitsUpdateMailbox, + BoxPipelineHandle, FinishPendingMergesAndShutdownPipeline, IndexingError, IndexingPipeline, + IndexingPipelineParams, IndexingService, Sequencer, SplitsUpdateMailbox, }; pub use crate::controlled_directory::ControlledDirectory; use crate::models::IndexingStatistics; diff --git a/quickwit/quickwit-indexing/src/models/mod.rs b/quickwit/quickwit-indexing/src/models/mod.rs index fd3188e2104..9dfdfde1594 100644 --- a/quickwit/quickwit-indexing/src/models/mod.rs +++ b/quickwit/quickwit-indexing/src/models/mod.rs @@ -21,9 +21,7 @@ mod merge_planner_message; mod merge_scratch; mod merge_statistics; mod packaged_split; -mod parquet_splits_update; mod processed_doc; -mod processed_parquet_batch; mod publish_lock; mod publisher_message; mod raw_doc_batch; @@ -42,9 +40,7 @@ pub use merge_planner_message::NewSplits; pub use merge_scratch::MergeScratch; pub use merge_statistics::MergeStatistics; pub use packaged_split::{PackagedSplit, PackagedSplitBatch}; -pub use parquet_splits_update::ParquetSplitsUpdate; pub use processed_doc::{ProcessedDoc, ProcessedDocBatch}; -pub use processed_parquet_batch::ProcessedParquetBatch; pub use publish_lock::{NewPublishLock, PublishLock}; pub use publisher_message::SplitsUpdate; use quickwit_proto::types::PublishToken; diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index f67dd65b218..40b9d16b54e 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -16,7 +16,7 @@ use std::fmt; use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_config::FileSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::metastore::SourceType; @@ -25,8 +25,7 @@ use quickwit_proto::types::SourceId; use super::doc_file_reader::ObjectUriBatchReader; #[cfg(feature = "queue-sources")] use super::queue_sources::coordinator::QueueCoordinator; -use crate::actors::Processor; -use crate::source::{Source, SourceContext, SourceRuntime, TypedSourceFactory}; +use crate::source::{ProcessorMailbox, Source, SourceContext, SourceRuntime, TypedSourceFactory}; enum FileSourceState { #[cfg(feature = "queue-sources")] @@ -51,12 +50,12 @@ impl fmt::Debug for FileSource { } #[async_trait] -impl Source

for FileSource { +impl Source for FileSource { #[allow(unused_variables)] async fn initialize( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { match &mut self.state { #[cfg(feature = "queue-sources")] @@ -70,8 +69,8 @@ impl Source

for FileSource { #[allow(unused_variables)] async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { match &mut self.state { #[cfg(feature = "queue-sources")] @@ -89,10 +88,10 @@ impl Source

for FileSource { *num_bytes_processed += batch_builder.num_bytes; *num_lines_processed += batch_builder.docs.len() as u64; processor_mailbox - .send_message(batch_builder.build()) + .send_raw_doc_batch(batch_builder.build(), ctx) .await?; if batch_reader.is_eof() { - ctx.send_exit_with_success(processor_mailbox).await?; + processor_mailbox.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } } @@ -108,7 +107,7 @@ impl Source

for FileSource { async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - ctx: &SourceContext

, + ctx: &SourceContext, ) -> anyhow::Result<()> { match &mut self.state { #[cfg(feature = "queue-sources")] @@ -215,7 +214,7 @@ mod tests { DUMMY_DOC, generate_dummy_doc_file, generate_index_doc_file, }; use crate::source::tests::SourceRuntimeBuilder; - use crate::source::{BATCH_NUM_BYTES_LIMIT, SourceActor}; + use crate::source::{BATCH_NUM_BYTES_LIMIT, ProcessorMailbox, SourceActor}; #[tokio::test] async fn test_file_source() { @@ -246,7 +245,7 @@ mod tests { .unwrap(); let file_source_actor = SourceActor { source: Box::new(file_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_file_source_mailbox, file_source_handle) = universe.spawn_builder().spawn(file_source_actor); @@ -299,7 +298,7 @@ mod tests { .unwrap(); let file_source_actor = SourceActor { source: Box::new(file_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_file_source_mailbox, file_source_handle) = universe.spawn_builder().spawn(file_source_actor); @@ -376,7 +375,7 @@ mod tests { .unwrap(); let file_source_actor = SourceActor { source: Box::new(file_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_file_source_mailbox, file_source_handle) = universe.spawn_builder().spawn(file_source_actor); @@ -412,13 +411,13 @@ mod localstack_tests { use super::*; use crate::actors::DocProcessor; use crate::models::RawDocBatch; - use crate::source::SourceActor; use crate::source::doc_file_reader::file_test_helpers::generate_dummy_doc_file; use crate::source::queue_sources::sqs_queue::test_helpers::{ create_queue, get_localstack_sqs_client, send_message, }; use crate::source::test_setup_helper::setup_index; use crate::source::tests::SourceRuntimeBuilder; + use crate::source::{ProcessorMailbox, SourceActor}; #[tokio::test] async fn test_file_source_sqs_notifications() { @@ -459,7 +458,7 @@ mod localstack_tests { { let actor = SourceActor { source: Box::new(sqs_source), - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox.clone()), }; let (_mailbox, handle) = universe.spawn_builder().spawn(actor); diff --git a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs index 4e4a67fa4b0..a44d4b95e19 100644 --- a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs +++ b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs @@ -22,7 +22,7 @@ use google_cloud_auth::credentials::CredentialsFile; use google_cloud_gax::retry::RetrySetting; use google_cloud_pubsub::client::{Client, ClientConfig}; use google_cloud_pubsub::subscription::Subscription; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_common::rand::append_random_suffix; use quickwit_config::PubSubSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; @@ -33,8 +33,9 @@ use tokio::time; use tracing::{debug, info, warn}; use super::{BATCH_NUM_BYTES_LIMIT, EMIT_BATCHES_TIMEOUT}; -use crate::actors::Processor; -use crate::source::{BatchBuilder, Source, SourceContext, SourceRuntime, TypedSourceFactory}; +use crate::source::{ + BatchBuilder, ProcessorMailbox, Source, SourceContext, SourceRuntime, TypedSourceFactory, +}; const DEFAULT_MAX_MESSAGES_PER_PULL: i32 = 1_000; @@ -153,11 +154,11 @@ impl GcpPubSubSource { } #[async_trait] -impl Source

for GcpPubSubSource { +impl Source for GcpPubSubSource { async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let now = Instant::now(); let mut batch_builder = BatchBuilder::new(SourceType::PubSub); @@ -191,7 +192,7 @@ impl Source

for GcpPubSubSource { // TODO: need to wait for all the id to be ack for at_least_once if self.should_exit() { info!(subscription=%self.subscription_name, "reached end of subscription"); - ctx.send_exit_with_success(processor_mailbox).await?; + processor_mailbox.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } if !batch_builder.checkpoint_delta.is_empty() { @@ -201,7 +202,7 @@ impl Source

for GcpPubSubSource { num_millis=%now.elapsed().as_millis(), "Sending doc batch to indexer."); let message = batch_builder.build(); - ctx.send_message(processor_mailbox, message).await?; + processor_mailbox.send_raw_doc_batch(message, ctx).await?; } Ok(Duration::default()) } @@ -209,7 +210,7 @@ impl Source

for GcpPubSubSource { async fn suggest_truncate( &mut self, _checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { // TODO: add ack of ids Ok(()) @@ -292,9 +293,10 @@ mod gcp_pubsub_emulator_tests { use serde_json::json; use super::*; + use crate::actors::DocProcessor; use crate::models::RawDocBatch; use crate::source::tests::SourceRuntimeBuilder; - use crate::source::{SourceActor, quickwit_supported_sources}; + use crate::source::{ProcessorMailbox, SourceActor, quickwit_supported_sources}; static GCP_TEST_PROJECT: &str = "quickwit-emulator"; @@ -388,10 +390,11 @@ mod gcp_pubsub_emulator_tests { let source_runtime = SourceRuntimeBuilder::new(index_uid, source_config).build(); let source = source_loader.load_source(source_runtime).await.unwrap(); - let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); + let (doc_processor_mailbox, doc_processor_inbox) = + universe.create_test_mailbox::(); let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); let (exit_status, exit_state) = source_handle.join().await; diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index b6709b2a76d..40e7fa6c896 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -20,7 +20,7 @@ use anyhow::Context; use async_trait::async_trait; use fnv::FnvHashMap; use itertools::Itertools; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_common::pubsub::EventBroker; use quickwit_common::retry::RetryParams; use quickwit_ingest::{ @@ -46,10 +46,9 @@ use tracing::{debug, error, info, warn}; use ulid::Ulid; use super::{ - BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, Source, SourceContext, - SourceRuntime, TypedSourceFactory, + BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, ProcessorMailbox, Source, + SourceContext, SourceRuntime, TypedSourceFactory, }; -use crate::actors::Processor; use crate::models::{LocalShardPositionsUpdate, NewPublishLock, NewPublishToken, PublishLock}; pub struct IngestSourceFactory; @@ -391,11 +390,11 @@ impl IngestSource { /// After this method has returned we are guaranteed to have the following post condition: /// - a alive publish lock / non-empty publish token /// - all currently assigned shards included in the `new_assigned_shard_ids` set. - async fn reset_if_needed( + async fn reset_if_needed( &mut self, new_assigned_shard_ids: &BTreeSet, - doc_processor_mailbox: &Mailbox, - ctx: &crate::source::SourceContext, + doc_processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> anyhow::Result<()> { // No need to do anything if the list of shards before and after are empty. if new_assigned_shard_ids.is_empty() && self.assigned_shards.is_empty() { @@ -441,26 +440,22 @@ impl IngestSource { self.publish_lock.kill().await; self.publish_lock = PublishLock::default(); self.publish_token = self.client_id.new_publish_token(); - ctx.send_message( - doc_processor_mailbox, - NewPublishLock(self.publish_lock.clone()), - ) - .await?; - ctx.send_message( - doc_processor_mailbox, - NewPublishToken(self.publish_token.clone()), - ) - .await?; + doc_processor_mailbox + .send_publish_lock(NewPublishLock(self.publish_lock.clone()), ctx) + .await?; + doc_processor_mailbox + .send_publish_token(NewPublishToken(self.publish_token.clone()), ctx) + .await?; Ok(()) } } #[async_trait] -impl Source

for IngestSource { +impl Source for IngestSource { async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let mut batch_builder = BatchBuilder::new(SourceType::IngestV2); @@ -502,7 +497,7 @@ impl Source

for IngestSource { "Sending doc batch to indexer." ); let message = batch_builder.build(); - ctx.send_message(processor_mailbox, message).await?; + processor_mailbox.send_raw_doc_batch(message, ctx).await?; } Ok(Duration::default()) } @@ -510,8 +505,8 @@ impl Source

for IngestSource { async fn assign_shards( &mut self, new_assigned_shard_ids: BTreeSet, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> anyhow::Result<()> { self.reset_if_needed(&new_assigned_shard_ids, processor_mailbox, ctx) .await?; @@ -621,7 +616,7 @@ impl Source

for IngestSource { async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { let truncate_up_to_positions: Vec<(ShardId, Position)> = checkpoint .iter() @@ -957,6 +952,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -966,7 +962,7 @@ mod tests { let shard_ids: BTreeSet = once(0).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); assert_eq!(sequence_rx.recv().await.unwrap(), 1); @@ -980,7 +976,7 @@ mod tests { let shard_ids: BTreeSet = (0..2).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); assert_eq!(sequence_rx.recv().await.unwrap(), 2); @@ -993,7 +989,7 @@ mod tests { let shard_ids: BTreeSet = (1..3).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); @@ -1162,6 +1158,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, _doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1171,7 +1168,7 @@ mod tests { BTreeSet::from_iter([ShardId::from(1), ShardId::from(2)]); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); @@ -1328,6 +1325,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, _doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1341,7 +1339,7 @@ mod tests { // In this scenario, the indexer will only be able to acquire shard 1. source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); @@ -1394,6 +1392,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1473,10 +1472,7 @@ mod tests { ); fetch_message_tx.send(Ok(in_flight_value)).await.unwrap(); - source - .emit_batches(&doc_processor_mailbox, &ctx) - .await - .unwrap(); + source.emit_batches(&processor_mailbox, &ctx).await.unwrap(); let doc_batch = doc_processor_inbox .recv_typed_message::() .await @@ -1502,10 +1498,7 @@ mod tests { assert_eq!(partition_deltas[1].1.from, Position::offset(22u64)); assert_eq!(partition_deltas[1].1.to, Position::eof(23u64)); - source - .emit_batches(&doc_processor_mailbox, &ctx) - .await - .unwrap(); + source.emit_batches(&processor_mailbox, &ctx).await.unwrap(); let shard = source.assigned_shards.get(&ShardId::from(2)).unwrap(); assert_eq!(shard.status, IndexingStatus::ReachedEof); @@ -1519,10 +1512,7 @@ mod tests { .await .unwrap(); - source - .emit_batches(&doc_processor_mailbox, &ctx) - .await - .unwrap(); + source.emit_batches(&processor_mailbox, &ctx).await.unwrap(); let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::Error); @@ -1543,10 +1533,7 @@ mod tests { ); fetch_message_tx.send(Ok(in_flight_value)).await.unwrap(); - source - .emit_batches(&doc_processor_mailbox, &ctx) - .await - .unwrap(); + source.emit_batches(&processor_mailbox, &ctx).await.unwrap(); let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::Active); } @@ -1629,6 +1616,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1636,14 +1624,11 @@ mod tests { let shard_ids: BTreeSet = BTreeSet::from_iter([ShardId::from(1)]); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); - source - .emit_batches(&doc_processor_mailbox, &ctx) - .await - .unwrap(); + source.emit_batches(&processor_mailbox, &ctx).await.unwrap(); let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::NotFound); @@ -1919,6 +1904,7 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, _doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox.clone()); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1933,7 +1919,7 @@ mod tests { }); source - .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) + .assign_shards(shard_ids, &processor_mailbox, &ctx) .await .unwrap(); diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index a1ad35b8cc6..7c2572db9d9 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -29,8 +29,7 @@ use serde::Serialize; use serde_json::Value as JsonValue; use tracing::{error, info}; -use super::{BatchBuilder, Source, SourceContext, TypedSourceFactory}; -use crate::actors::Processor; +use super::{BatchBuilder, ProcessorMailbox, Source, SourceContext, TypedSourceFactory}; use crate::source::SourceRuntime; /// Wait time for SourceActor before pooling for new documents. @@ -114,10 +113,10 @@ impl IngestApiSource { Ok(ingest_api_source) } - async fn send_suggest_truncate_to_ingest_service( + async fn send_suggest_truncate_to_ingest_service( &self, up_to_position_included: u64, - ctx: &crate::source::SourceContext, + ctx: &SourceContext, ) -> anyhow::Result<()> { let suggest_truncate_req = SuggestTruncateRequest { index_id: self.source_runtime.index_id().to_string(), @@ -136,11 +135,11 @@ impl IngestApiSource { } #[async_trait] -impl Source

for IngestApiSource { +impl Source for IngestApiSource { async fn initialize( &mut self, - _: &Mailbox

, - ctx: &SourceContext

, + _: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { if let Some(position) = self.counters.previous_offset { self.send_suggest_truncate_to_ingest_service(position, ctx) @@ -151,8 +150,8 @@ impl Source

for IngestApiSource { async fn emit_batches( &mut self, - batch_sink: &Mailbox

, - ctx: &SourceContext

, + batch_sink: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let fetch_req = FetchRequest { index_id: self.source_runtime.index_id().to_string(), @@ -202,14 +201,16 @@ impl Source

for IngestApiSource { .map_err(anyhow::Error::from)?; self.update_counters(current_offset, batch_builder.docs.len() as u64); - ctx.send_message(batch_sink, batch_builder.build()).await?; + batch_sink + .send_raw_doc_batch(batch_builder.build(), ctx) + .await?; Ok(Duration::default()) } async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - ctx: &SourceContext

, + ctx: &SourceContext, ) -> anyhow::Result<()> { if let Some(Position::Offset(offset)) = checkpoint.position_for_partition(&self.partition_id) @@ -263,8 +264,8 @@ mod tests { use super::*; use crate::actors::DocProcessor; use crate::models::RawDocBatch; - use crate::source::SourceActor; use crate::source::tests::SourceRuntimeBuilder; + use crate::source::{ProcessorMailbox, SourceActor}; fn make_ingest_request( index_id: IndexId, @@ -321,7 +322,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); @@ -431,7 +432,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); @@ -484,7 +485,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); @@ -536,7 +537,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); @@ -601,7 +602,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); @@ -698,7 +699,7 @@ mod tests { let ingest_api_source = IngestApiSource::try_new(source_runtime).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (ingest_api_source_mailbox, ingest_api_source_handle) = universe.spawn_builder().spawn(ingest_api_source_actor); diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index f8f4c73fb4d..5b1c3081b6d 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -21,7 +21,7 @@ use async_trait::async_trait; use bytes::Bytes; use itertools::Itertools; use oneshot; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_config::KafkaSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::metastore::SourceType; @@ -40,11 +40,10 @@ use tokio::task::{JoinHandle, spawn_blocking}; use tokio::time; use tracing::{debug, info, warn}; -use crate::actors::Processor; use crate::models::{NewPublishLock, PublishLock}; use crate::source::{ - BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, Source, SourceContext, - SourceRuntime, TypedSourceFactory, + BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, ProcessorMailbox, Source, + SourceContext, SourceRuntime, TypedSourceFactory, }; type GroupId = String; @@ -333,9 +332,9 @@ impl KafkaSource { Ok(()) } - async fn process_assign_partitions( + async fn process_assign_partitions( &mut self, - ctx: &crate::source::SourceContext, + ctx: &SourceContext, partitions: &[i32], assignment_tx: oneshot::Sender>, ) -> anyhow::Result<()> { @@ -391,10 +390,10 @@ impl KafkaSource { Ok(()) } - async fn process_revoke_partitions( + async fn process_revoke_partitions( &mut self, - ctx: &crate::source::SourceContext, - doc_processor_mailbox: &Mailbox, + ctx: &SourceContext, + doc_processor_mailbox: &ProcessorMailbox, batch: &mut BatchBuilder, ack_tx: oneshot::Sender<()>, ) -> anyhow::Result<()> { @@ -406,11 +405,9 @@ impl KafkaSource { batch.clear(); self.publish_lock = PublishLock::default(); self.state.num_rebalances += 1; - ctx.send_message( - doc_processor_mailbox, - NewPublishLock(self.publish_lock.clone()), - ) - .await?; + doc_processor_mailbox + .send_publish_lock(NewPublishLock(self.publish_lock.clone()), ctx) + .await?; Ok(()) } @@ -441,22 +438,23 @@ impl KafkaSource { } #[async_trait] -impl Source

for KafkaSource { +impl Source for KafkaSource { async fn initialize( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { let publish_lock = self.publish_lock.clone(); - ctx.send_message(processor_mailbox, NewPublishLock(publish_lock)) + processor_mailbox + .send_publish_lock(NewPublishLock(publish_lock), ctx) .await?; Ok(()) } async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let now = Instant::now(); let mut batch_builder = BatchBuilder::new(SourceType::Kafka); @@ -492,11 +490,11 @@ impl Source

for KafkaSource { "sending doc batch to indexer" ); let message = batch_builder.build(); - ctx.send_message(processor_mailbox, message).await?; + processor_mailbox.send_raw_doc_batch(message, ctx).await?; } if self.should_exit() { info!(topic = %self.topic, "reached end of topic"); - ctx.send_exit_with_success(processor_mailbox).await?; + processor_mailbox.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } Ok(Duration::default()) @@ -505,7 +503,7 @@ impl Source

for KafkaSource { async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { self.truncate(checkpoint)?; Ok(()) @@ -514,7 +512,7 @@ impl Source

for KafkaSource { async fn finalize( &mut self, _exit_status: &ActorExitStatus, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { self.poll_loop_jh.abort(); Ok(()) @@ -772,9 +770,10 @@ mod kafka_broker_tests { use tokio::sync::watch; use super::*; + use crate::actors::DocProcessor; use crate::source::test_setup_helper::setup_index; use crate::source::tests::SourceRuntimeBuilder; - use crate::source::{RawDocBatch, SourceActor, quickwit_supported_sources}; + use crate::source::{ProcessorMailbox, RawDocBatch, SourceActor, quickwit_supported_sources}; fn create_base_consumer(group_id: &str) -> BaseConsumer { ClientConfig::new() @@ -1110,7 +1109,7 @@ mod kafka_broker_tests { let universe = Universe::with_accelerated_time(); let (source_mailbox, _source_inbox) = universe.create_test_mailbox(); - let (indexer_mailbox, indexer_inbox) = universe.create_test_mailbox(); + let (indexer_mailbox, indexer_inbox) = universe.create_test_mailbox::(); let (observable_state_tx, _observable_state_rx) = watch::channel(json!({})); let ctx: ActorContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); @@ -1123,8 +1122,9 @@ mod kafka_broker_tests { assert!(publish_lock.is_alive()); assert_eq!(kafka_source.state.num_rebalances, 0); + let processor_mailbox = ProcessorMailbox::new(indexer_mailbox); kafka_source - .process_revoke_partitions(&ctx, &indexer_mailbox, &mut batch_builder, ack_tx) + .process_revoke_partitions(&ctx, &processor_mailbox, &mut batch_builder, ack_tx) .await .unwrap(); @@ -1272,10 +1272,11 @@ mod kafka_broker_tests { .with_metastore(metastore) .build(); let source = source_loader.load_source(source_runtime).await?; - let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); + let (doc_processor_mailbox, doc_processor_inbox) = + universe.create_test_mailbox::(); let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); let (exit_status, exit_state) = source_handle.join().await; @@ -1323,11 +1324,12 @@ mod kafka_broker_tests { let source_runtime = SourceRuntimeBuilder::new(index_uid, source_config) .with_metastore(metastore) .build(); - let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); + let (doc_processor_mailbox, doc_processor_inbox) = + universe.create_test_mailbox::(); let source = source_loader.load_source(source_runtime).await?; let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); let (exit_status, exit_state) = source_handle.join().await; @@ -1398,10 +1400,11 @@ mod kafka_broker_tests { .with_metastore(metastore) .build(); let source = source_loader.load_source(source_runtime).await?; - let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); + let (doc_processor_mailbox, doc_processor_inbox) = + universe.create_test_mailbox::(); let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); let (exit_status, exit_state) = source_handle.join().await; @@ -1451,10 +1454,11 @@ mod kafka_broker_tests { .with_metastore(metastore) .build(); let source = source_loader.load_source(source_runtime).await?; - let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); + let (doc_processor_mailbox, doc_processor_inbox) = + universe.create_test_mailbox::(); let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); let (exit_status, exit_state) = source_handle.join().await; diff --git a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs index 8d6e7e662b7..d9e58647a4f 100644 --- a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs +++ b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs @@ -21,7 +21,7 @@ use async_trait::async_trait; use aws_sdk_kinesis::Client as KinesisClient; use bytes::Bytes; use itertools::Itertools; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_aws::get_aws_config; use quickwit_common::retry::RetryParams; use quickwit_config::{KinesisSourceParams, RegionOrEndpoint}; @@ -35,11 +35,10 @@ use tracing::{info, warn}; use super::api::list_shards; use super::shard_consumer::{ShardConsumer, ShardConsumerHandle, ShardConsumerMessage}; -use crate::actors::Processor; use crate::source::kinesis::helpers::get_kinesis_client; use crate::source::{ - BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, Source, SourceContext, - SourceRuntime, TypedSourceFactory, + BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, ProcessorMailbox, Source, + SourceContext, SourceRuntime, TypedSourceFactory, }; type ShardId = String; @@ -131,9 +130,9 @@ impl KinesisSource { Ok(kinesis_source) } - fn spawn_shard_consumer( + fn spawn_shard_consumer( &mut self, - ctx: &SourceContext, + ctx: &SourceContext, shard_id: ShardId, checkpoint: &SourceCheckpoint, ) { @@ -185,11 +184,11 @@ impl KinesisSource { } #[async_trait] -impl Source

for KinesisSource { +impl Source for KinesisSource { async fn initialize( &mut self, - _processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + _processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { let shards = ctx .protect_future(list_shards( @@ -218,8 +217,8 @@ impl Source

for KinesisSource { async fn emit_batches( &mut self, - indexer_mailbox: &Mailbox

, - ctx: &SourceContext

, + indexer_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let mut batch_builder = BatchBuilder::new(SourceType::Kinesis); let deadline = time::sleep(*EMIT_BATCHES_TIMEOUT); @@ -313,12 +312,13 @@ impl Source

for KinesisSource { self.state.num_records_processed += batch_builder.docs.len() as u64; if !batch_builder.checkpoint_delta.is_empty() { - ctx.send_message(indexer_mailbox, batch_builder.build()) + indexer_mailbox + .send_raw_doc_batch(batch_builder.build(), ctx) .await?; } if self.state.shard_consumers.is_empty() { info!(stream_name = %self.stream_name, "reached end of stream"); - ctx.send_exit_with_success(indexer_mailbox).await?; + indexer_mailbox.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } Ok(Duration::default()) @@ -377,11 +377,11 @@ mod tests { use super::*; use crate::actors::DocProcessor; use crate::models::RawDocBatch; - use crate::source::SourceActor; use crate::source::kinesis::helpers::tests::{ make_shard_id, put_records_into_shards, setup, teardown, }; use crate::source::tests::SourceRuntimeBuilder; + use crate::source::{ProcessorMailbox, SourceActor}; // Sequence number type SeqNo = String; @@ -445,7 +445,7 @@ mod tests { let actor = SourceActor { source: Box::new(kinesis_source), - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox.clone()), }; let (_mailbox, handle) = universe.spawn_builder().spawn(actor); let (exit_status, _exit_state) = handle.join().await; @@ -480,7 +480,7 @@ mod tests { .unwrap(); let actor = SourceActor { source: Box::new(kinesis_source), - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox.clone()), }; let (_mailbox, handle) = universe.spawn_builder().spawn(actor); let (exit_status, exit_state) = handle.join().await; @@ -533,7 +533,7 @@ mod tests { .unwrap(); let actor = SourceActor { source: Box::new(kinesis_source), - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox.clone()), }; let (_mailbox, handle) = universe.spawn_builder().spawn(actor); let (exit_status, exit_state) = handle.join().await; @@ -603,7 +603,7 @@ mod tests { .unwrap(); let actor = SourceActor { source: Box::new(kinesis_source), - processor_mailbox: doc_processor_mailbox.clone(), + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox.clone()), }; let (_mailbox, handle) = universe.spawn_builder().spawn(actor); let (exit_status, exit_state) = handle.join().await; diff --git a/quickwit/quickwit-indexing/src/source/kinesis/shard_consumer.rs b/quickwit/quickwit-indexing/src/source/kinesis/shard_consumer.rs index df00f963588..45eb4c05787 100644 --- a/quickwit/quickwit-indexing/src/source/kinesis/shard_consumer.rs +++ b/quickwit/quickwit-indexing/src/source/kinesis/shard_consumer.rs @@ -103,7 +103,7 @@ impl ShardConsumer { } } - pub fn spawn(self, ctx: &SourceContext

) -> ShardConsumerHandle { + pub fn spawn(self, ctx: &SourceContext) -> ShardConsumerHandle { let (_mailbox, _actor_handle) = ctx.spawn_actor().spawn(self); ShardConsumerHandle { _mailbox, diff --git a/quickwit/quickwit-indexing/src/source/mod.rs b/quickwit/quickwit-indexing/src/source/mod.rs index 86d40e5288f..e19327f012b 100644 --- a/quickwit/quickwit-indexing/src/source/mod.rs +++ b/quickwit/quickwit-indexing/src/source/mod.rs @@ -62,6 +62,7 @@ mod ingest_api_source; mod kafka_source; #[cfg(feature = "kinesis")] mod kinesis; +mod processor_mailbox; #[cfg(feature = "pulsar")] mod pulsar_source; #[cfg(feature = "queue-sources")] @@ -86,11 +87,12 @@ pub use kafka_source::{KafkaSource, KafkaSourceFactory}; #[cfg(feature = "kinesis")] pub use kinesis::kinesis_source::{KinesisSource, KinesisSourceFactory}; use once_cell::sync::{Lazy, OnceCell}; +pub use processor_mailbox::ProcessorMailbox; #[cfg(feature = "pulsar")] pub use pulsar_source::{PulsarSource, PulsarSourceFactory}; #[cfg(feature = "sqs")] pub use queue_sources::sqs_queue; -use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox}; +use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; use quickwit_common::metrics::{GaugeGuard, MEMORY_METRICS}; use quickwit_common::pubsub::EventBroker; use quickwit_common::runtimes::RuntimeType; @@ -116,17 +118,10 @@ pub use void_source::{VoidSource, VoidSourceFactory}; use self::doc_file_reader::dir_and_filename; use self::stdin_source::StdinSourceFactory; -use crate::actors::{DocProcessor, ParquetDocProcessor, Processor}; use crate::models::RawDocBatch; use crate::source::ingest::IngestSourceFactory; use crate::source::ingest_api_source::IngestApiSourceFactory; -/// Type alias for SourceContext with ParquetDocProcessor (for metrics pipeline). -pub type ParquetSourceContext = SourceContext; - -/// Type alias for SourceLoader with ParquetDocProcessor (for metrics pipeline). -pub type ParquetSourceLoader = SourceLoader; - /// Number of bytes after which we cut a new batch. /// /// We try to emit chewable batches for the indexer. @@ -213,9 +208,7 @@ impl SourceRuntime { } } -pub type SourceContext

= ActorContext>; - -pub type LogsSourceContext = SourceContext; +pub type SourceContext = ActorContext; /// A Source is a trait that is mounted in a light wrapping Actor called `SourceActor`. /// @@ -241,12 +234,12 @@ pub type LogsSourceContext = SourceContext; /// } /// ``` #[async_trait] -pub trait Source: Send + 'static { +pub trait Source: Send + 'static { /// This method will be called before any calls to `emit_batches`. async fn initialize( &mut self, - _processor_mailbox: &Mailbox

, - _ctx: &SourceContext

, + _processor_mailbox: &ProcessorMailbox, + _ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { Ok(()) } @@ -260,8 +253,8 @@ pub trait Source: Send + 'static { /// should wait before polling again. async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result; /// Assign shards is called when the source is assigned a new set of shards by the control @@ -269,8 +262,8 @@ pub trait Source: Send + 'static { async fn assign_shards( &mut self, _shard_ids: BTreeSet, - _processor_mailbox: &Mailbox

, - _ctx: &SourceContext

, + _processor_mailbox: &ProcessorMailbox, + _ctx: &SourceContext, ) -> anyhow::Result<()> { Ok(()) } @@ -292,7 +285,7 @@ pub trait Source: Send + 'static { async fn suggest_truncate( &mut self, _checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { Ok(()) } @@ -301,7 +294,7 @@ pub trait Source: Send + 'static { async fn finalize( &mut self, _exit_status: &ActorExitStatus, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { Ok(()) } @@ -319,9 +312,9 @@ pub trait Source: Send + 'static { /// The SourceActor acts as a thin wrapper over a source trait object to execute. /// /// It mostly takes care of running a loop calling `emit_batches(...)`. -pub struct SourceActor { - pub source: Box>, - pub processor_mailbox: Mailbox

, +pub struct SourceActor { + pub source: Box, + pub processor_mailbox: ProcessorMailbox, } #[derive(Debug)] @@ -336,7 +329,7 @@ pub struct Assignment { pub struct AssignShards(pub Assignment); #[async_trait] -impl Actor for SourceActor

{ +impl Actor for SourceActor { type ObservableState = JsonValue; fn name(&self) -> String { @@ -355,7 +348,7 @@ impl Actor for SourceActor

{ false } - async fn initialize(&mut self, ctx: &SourceContext

) -> Result<(), ActorExitStatus> { + async fn initialize(&mut self, ctx: &SourceContext) -> Result<(), ActorExitStatus> { self.source.initialize(&self.processor_mailbox, ctx).await?; self.handle(Loop, ctx).await?; Ok(()) @@ -364,7 +357,7 @@ impl Actor for SourceActor

{ async fn finalize( &mut self, exit_status: &ActorExitStatus, - ctx: &SourceContext

, + ctx: &SourceContext, ) -> anyhow::Result<()> { self.source.finalize(exit_status, ctx).await?; Ok(()) @@ -372,14 +365,10 @@ impl Actor for SourceActor

{ } #[async_trait] -impl Handler for SourceActor

{ +impl Handler for SourceActor { type Reply = (); - async fn handle( - &mut self, - _message: Loop, - ctx: &SourceContext

, - ) -> Result<(), ActorExitStatus> { + async fn handle(&mut self, _message: Loop, ctx: &SourceContext) -> Result<(), ActorExitStatus> { let wait_for = self .source .emit_batches(&self.processor_mailbox, ctx) @@ -394,13 +383,13 @@ impl Handler for SourceActor

{ } #[async_trait] -impl Handler for SourceActor

{ +impl Handler for SourceActor { type Reply = (); async fn handle( &mut self, assign_shards_message: AssignShards, - ctx: &SourceContext

, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { let AssignShards(Assignment { shard_ids }) = assign_shards_message; self.source @@ -433,24 +422,6 @@ pub fn quickwit_supported_sources() -> &'static SourceLoader { }) } -/// Returns the source loader for parquet pipelines (ParquetDocProcessor). -/// -/// Metrics pipelines currently only support IngestV2 sources, which is the -/// production source type for metrics ingestion. -pub fn quickwit_supported_parquet_sources() -> &'static ParquetSourceLoader { - static PARQUET_SOURCE_LOADER: OnceCell = OnceCell::new(); - PARQUET_SOURCE_LOADER.get_or_init(|| { - let mut source_factory = ParquetSourceLoader::default(); - // Only IngestV2 is currently used for metrics ingestion - source_factory.add_source(SourceType::IngestV2, IngestSourceFactory); - // Add other sources for testing/development - source_factory.add_source(SourceType::File, FileSourceFactory); - source_factory.add_source(SourceType::Vec, VecSourceFactory); - source_factory.add_source(SourceType::Void, VoidSourceFactory); - source_factory - }) -} - pub async fn check_source_connectivity( storage_resolver: &StorageResolver, source_config: &SourceConfig, @@ -516,13 +487,13 @@ pub async fn check_source_connectivity( pub struct SuggestTruncate(pub SourceCheckpoint); #[async_trait] -impl Handler for SourceActor

{ +impl Handler for SourceActor { type Reply = (); async fn handle( &mut self, suggest_truncate: SuggestTruncate, - ctx: &SourceContext

, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { let SuggestTruncate(checkpoint) = suggest_truncate; diff --git a/quickwit/quickwit-indexing/src/source/processor_mailbox.rs b/quickwit/quickwit-indexing/src/source/processor_mailbox.rs new file mode 100644 index 00000000000..00372669239 --- /dev/null +++ b/quickwit/quickwit-indexing/src/source/processor_mailbox.rs @@ -0,0 +1,134 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Type-erased mailbox for sending messages to any doc processor actor. +//! +//! This decouples `Source` implementations from the specific processor actor +//! type (e.g. `DocProcessor` for logs, `ParquetDocProcessor` for metrics). + +use std::sync::Arc; + +use async_trait::async_trait; +use quickwit_actors::{Actor, Command, DeferableReplyHandler, Mailbox, SendError}; + +use super::SourceContext; +use crate::models::{NewPublishLock, NewPublishToken, RawDocBatch}; + +/// Internal trait used to type-erase the concrete `Mailbox`. +#[async_trait] +trait ProcessorMailboxInner: Send + Sync + 'static { + async fn send_raw_doc_batch(&self, batch: RawDocBatch) -> Result<(), SendError>; + async fn send_publish_lock(&self, lock: NewPublishLock) -> Result<(), SendError>; + async fn send_publish_token(&self, token: NewPublishToken) -> Result<(), SendError>; + async fn send_exit_with_success(&self) -> Result<(), SendError>; +} + +/// Wraps a concrete `Mailbox` for any actor A that handles the three +/// processor messages (`RawDocBatch`, `NewPublishLock`, `NewPublishToken`). +struct TypedProcessorMailbox +where A: Actor + + DeferableReplyHandler + + DeferableReplyHandler + + DeferableReplyHandler +{ + mailbox: Mailbox, +} + +#[async_trait] +impl ProcessorMailboxInner for TypedProcessorMailbox +where A: Actor + + DeferableReplyHandler + + DeferableReplyHandler + + DeferableReplyHandler +{ + async fn send_raw_doc_batch(&self, batch: RawDocBatch) -> Result<(), SendError> { + self.mailbox.send_message(batch).await?; + Ok(()) + } + + async fn send_publish_lock(&self, lock: NewPublishLock) -> Result<(), SendError> { + self.mailbox.send_message(lock).await?; + Ok(()) + } + + async fn send_publish_token(&self, token: NewPublishToken) -> Result<(), SendError> { + self.mailbox.send_message(token).await?; + Ok(()) + } + + async fn send_exit_with_success(&self) -> Result<(), SendError> { + self.mailbox.send_message(Command::ExitWithSuccess).await?; + Ok(()) + } +} + +/// Type-erased mailbox for sending messages to any doc processor actor. +/// +/// This decouples `Source` implementations from the specific processor actor +/// type (e.g. `DocProcessor` for logs, `ParquetDocProcessor` for metrics). +#[derive(Clone)] +pub struct ProcessorMailbox { + inner: Arc, +} + +impl ProcessorMailbox { + /// Create a `ProcessorMailbox` from any actor mailbox whose actor implements + /// the required message handlers. + pub fn new(mailbox: Mailbox) -> Self + where A: Actor + + DeferableReplyHandler + + DeferableReplyHandler + + DeferableReplyHandler { + Self { + inner: Arc::new(TypedProcessorMailbox { mailbox }), + } + } + + /// Send a `RawDocBatch` to the processor. + /// + /// The source context's protect zone is held while the send is in flight, + /// so the supervisor does not consider the source actor stuck while waiting + /// on backpressure from the processor mailbox. + pub async fn send_raw_doc_batch( + &self, + batch: RawDocBatch, + ctx: &SourceContext, + ) -> Result<(), SendError> { + let _guard = ctx.protect_zone(); + self.inner.send_raw_doc_batch(batch).await + } + + pub async fn send_publish_lock( + &self, + lock: NewPublishLock, + ctx: &SourceContext, + ) -> Result<(), SendError> { + let _guard = ctx.protect_zone(); + self.inner.send_publish_lock(lock).await + } + + pub async fn send_publish_token( + &self, + token: NewPublishToken, + ctx: &SourceContext, + ) -> Result<(), SendError> { + let _guard = ctx.protect_zone(); + self.inner.send_publish_token(token).await + } + + pub async fn send_exit_with_success(&self, ctx: &SourceContext) -> Result<(), SendError> { + let _guard = ctx.protect_zone(); + self.inner.send_exit_with_success().await + } +} diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index ce02d857b78..cd62c27f314 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -26,7 +26,7 @@ use pulsar::message::proto::MessageIdData; use pulsar::{ Authentication, Consumer, DeserializeMessage, Payload, Pulsar, SubType, TokioExecutor, }; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_config::{PulsarSourceAuth, PulsarSourceParams}; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::metastore::SourceType; @@ -35,10 +35,9 @@ use serde_json::{Value as JsonValue, json}; use tokio::time; use tracing::{debug, info, warn}; -use crate::actors::Processor; use crate::source::{ - BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, Source, SourceContext, - SourceRuntime, TypedSourceFactory, + BATCH_NUM_BYTES_LIMIT, BatchBuilder, EMIT_BATCHES_TIMEOUT, ProcessorMailbox, Source, + SourceContext, SourceRuntime, TypedSourceFactory, }; type PulsarConsumer = Consumer; @@ -210,11 +209,11 @@ impl PulsarSource { } #[async_trait] -impl Source

for PulsarSource { +impl Source for PulsarSource { async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let now = Instant::now(); let mut batch_builder = BatchBuilder::new(SourceType::Pulsar); @@ -252,7 +251,7 @@ impl Source

for PulsarSource { "sending doc batch to indexer" ); let message = batch_builder.build(); - ctx.send_message(processor_mailbox, message).await?; + processor_mailbox.send_raw_doc_batch(message, ctx).await?; } Ok(Duration::default()) } @@ -260,7 +259,7 @@ impl Source

for PulsarSource { async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { self.try_ack_messages(checkpoint).await } @@ -272,7 +271,7 @@ impl Source

for PulsarSource { async fn finalize( &mut self, _exit_status: &ActorExitStatus, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { self.pulsar_consumer.close().await?; Ok(()) @@ -458,7 +457,9 @@ mod pulsar_broker_tests { use crate::source::pulsar_source::{msg_id_from_position, msg_id_to_position}; use crate::source::test_setup_helper::setup_index; use crate::source::tests::SourceRuntimeBuilder; - use crate::source::{RawDocBatch, SourceActor, SuggestTruncate, quickwit_supported_sources}; + use crate::source::{ + ProcessorMailbox, RawDocBatch, SourceActor, SuggestTruncate, quickwit_supported_sources, + }; static PULSAR_URI: &str = "pulsar://localhost:6650"; static PULSAR_ADMIN_URI: &str = "http://localhost:8081"; @@ -652,7 +653,7 @@ mod pulsar_broker_tests { let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox(); let source_actor = SourceActor { source, - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_source_mailbox, source_handle) = universe.spawn_builder().spawn(source_actor); diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs b/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs index 40919879e69..6ae1f05f06b 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/coordinator.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use std::time::Duration; use itertools::Itertools; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_common::rate_limited_error; use quickwit_config::{FileSourceMessageType, FileSourceSqs}; use quickwit_metastore::checkpoint::SourceCheckpoint; @@ -34,9 +34,8 @@ use super::local_state::QueueLocalState; use super::message::{MessageType, PreProcessingError, ReadyMessage}; use super::shared_state::{QueueSharedState, checkpoint_messages}; use super::visibility::{VisibilitySettings, spawn_visibility_task}; -use crate::actors::Processor; use crate::models::{NewPublishLock, NewPublishToken, PublishLock}; -use crate::source::{SourceContext, SourceRuntime}; +use crate::source::{ProcessorMailbox, SourceContext, SourceRuntime}; /// Maximum duration that the `emit_batches()` callback can wait for /// `queue.receive()` calls. If too small, the actor loop will spin @@ -146,27 +145,23 @@ impl QueueCoordinator { )) } - pub async fn initialize( + pub async fn initialize( &mut self, - doc_processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + doc_processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result<(), ActorExitStatus> { let publish_lock = self.publish_lock.clone(); - ctx.send_message(doc_processor_mailbox, NewPublishLock(publish_lock)) + doc_processor_mailbox + .send_publish_lock(NewPublishLock(publish_lock), ctx) + .await?; + doc_processor_mailbox + .send_publish_token(NewPublishToken(self.publish_token.clone()), ctx) .await?; - ctx.send_message( - doc_processor_mailbox, - NewPublishToken(self.publish_token.clone()), - ) - .await?; Ok(()) } /// Polls messages from the queue and prepares them for processing - async fn poll_messages( - &mut self, - ctx: &SourceContext

, - ) -> Result<(), ActorExitStatus> { + async fn poll_messages(&mut self, ctx: &SourceContext) -> Result<(), ActorExitStatus> { let raw_messages = self .queue_receiver .receive(1, self.visibility_settings.deadline_for_receive) @@ -252,10 +247,10 @@ impl QueueCoordinator { Ok(()) } - pub async fn emit_batches( + pub async fn emit_batches( &mut self, - doc_processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + doc_processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { if let Some(in_progress_ref) = self.local_state.read_in_progress_mut() { // TODO: should we kill the publish lock if the message visibility extension failed? @@ -266,7 +261,7 @@ impl QueueCoordinator { self.observable_state.num_lines_processed += batch_builder.docs.len() as u64; self.observable_state.num_bytes_processed += batch_builder.num_bytes; doc_processor_mailbox - .send_message(batch_builder.build()) + .send_raw_doc_batch(batch_builder.build(), ctx) .await?; if in_progress_ref.batch_reader.is_eof() { self.local_state.drop_currently_read().await?; @@ -293,10 +288,10 @@ impl QueueCoordinator { Ok(Duration::ZERO) } - pub async fn suggest_truncate( + pub async fn suggest_truncate( &mut self, checkpoint: SourceCheckpoint, - _ctx: &SourceContext

, + _ctx: &SourceContext, ) -> anyhow::Result<()> { let committed_partition_ids = checkpoint .iter() @@ -335,7 +330,7 @@ mod tests { use crate::source::queue_sources::memory_queue::MemoryQueueForTests; use crate::source::queue_sources::message::PreProcessedPayload; use crate::source::queue_sources::shared_state::shared_state_for_tests::init_state; - use crate::source::{BATCH_NUM_BYTES_LIMIT, SourceActor}; + use crate::source::{BATCH_NUM_BYTES_LIMIT, ProcessorMailbox, SourceActor}; fn setup_coordinator( queue: Arc, @@ -375,17 +370,18 @@ mod tests { let (source_mailbox, _source_inbox) = universe.create_test_mailbox::(); let (doc_processor_mailbox, doc_processor_inbox) = universe.create_test_mailbox::(); + let processor_mailbox = ProcessorMailbox::new(doc_processor_mailbox); let (observable_state_tx, _observable_state_rx) = watch::channel(serde_json::Value::Null); let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); coordinator - .initialize(&doc_processor_mailbox, &ctx) + .initialize(&processor_mailbox, &ctx) .await .unwrap(); coordinator - .emit_batches(&doc_processor_mailbox, &ctx) + .emit_batches(&processor_mailbox, &ctx) .await .unwrap(); @@ -398,7 +394,7 @@ mod tests { // future. for _ in 0..(messages.len() * 4) { coordinator - .emit_batches(&doc_processor_mailbox, &ctx) + .emit_batches(&processor_mailbox, &ctx) .await .unwrap(); } diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs b/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs index 8f166d96de0..3af4deaa278 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/visibility.rs @@ -23,7 +23,6 @@ use quickwit_actors::{ use serde_json::{Value as JsonValue, json}; use super::Queue; -use crate::actors::Processor; use crate::source::SourceContext; #[derive(Debug, Clone)] @@ -81,8 +80,8 @@ pub(super) struct VisibilityTaskHandle { /// (represented by its ack_id) is extended when required. We prefer applying /// ample margins in the extension process to avoid missing deadlines while also /// keeping the number of extension requests (and associated cost) small. -pub(super) fn spawn_visibility_task( - ctx: &SourceContext

, +pub(super) fn spawn_visibility_task( + ctx: &SourceContext, queue: Arc, ack_id: String, current_deadline: Instant, diff --git a/quickwit/quickwit-indexing/src/source/source_factory.rs b/quickwit/quickwit-indexing/src/source/source_factory.rs index dd6ebbfc913..1a0f1273861 100644 --- a/quickwit/quickwit-indexing/src/source/source_factory.rs +++ b/quickwit/quickwit-indexing/src/source/source_factory.rs @@ -21,20 +21,17 @@ use quickwit_proto::types::SourceId; use thiserror::Error; use super::Source; -use crate::actors::{DocProcessor, ParquetDocProcessor, Processor}; use crate::source::SourceRuntime; #[async_trait] -pub trait SourceFactory: Send + Sync + 'static { - async fn create_source( - &self, - source_runtime: SourceRuntime, - ) -> anyhow::Result>>; +pub trait SourceFactory: Send + Sync + 'static { + async fn create_source(&self, source_runtime: SourceRuntime) + -> anyhow::Result>; } #[async_trait] pub trait TypedSourceFactory: Send + Sync + 'static { - type Source: Source + Source; + type Source: Source; type Params: serde::de::DeserializeOwned + Send + Sync + 'static; async fn typed_create_source( @@ -56,24 +53,11 @@ impl SourceFactory for T { } } -#[async_trait] -impl SourceFactory for T { - async fn create_source( - &self, - source_runtime: SourceRuntime, - ) -> anyhow::Result>> { - let typed_params: T::Params = - serde_json::from_value(source_runtime.source_config.params())?; - let source = Self::typed_create_source(source_runtime, typed_params).await?; - Ok(Box::new(source)) - } -} - -pub struct SourceLoader { - type_to_factory: HashMap>>, +pub struct SourceLoader { + type_to_factory: HashMap>, } -impl Default for SourceLoader

{ +impl Default for SourceLoader { fn default() -> Self { Self { type_to_factory: HashMap::default(), @@ -100,8 +84,8 @@ pub enum SourceLoaderError { }, } -impl SourceLoader

{ - pub fn add_source>(&mut self, source_type: SourceType, source_factory: F) { +impl SourceLoader { + pub fn add_source(&mut self, source_type: SourceType, source_factory: F) { self.type_to_factory .insert(source_type, Box::new(source_factory)); } @@ -109,7 +93,7 @@ impl SourceLoader

{ pub async fn load_source( &self, source_runtime: SourceRuntime, - ) -> Result>, SourceLoaderError> { + ) -> Result, SourceLoaderError> { let source_type = source_runtime.source_config.source_type(); let source_id = source_runtime.source_id().to_string(); let source_factory = self.type_to_factory.get(&source_type).ok_or_else(|| { diff --git a/quickwit/quickwit-indexing/src/source/stdin_source.rs b/quickwit/quickwit-indexing/src/source/stdin_source.rs index 72327df18e3..08c619c59cc 100644 --- a/quickwit/quickwit-indexing/src/source/stdin_source.rs +++ b/quickwit/quickwit-indexing/src/source/stdin_source.rs @@ -16,14 +16,13 @@ use std::fmt; use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_common::Progress; use quickwit_proto::metastore::SourceType; use tokio::io::{AsyncBufReadExt, BufReader}; use super::{BATCH_NUM_BYTES_LIMIT, BatchBuilder}; -use crate::actors::Processor; -use crate::source::{Source, SourceContext, SourceRuntime, TypedSourceFactory}; +use crate::source::{ProcessorMailbox, Source, SourceContext, SourceRuntime, TypedSourceFactory}; pub struct StdinBatchReader { reader: BufReader, @@ -76,20 +75,20 @@ impl fmt::Debug for StdinSource { } #[async_trait] -impl Source

for StdinSource { +impl Source for StdinSource { async fn emit_batches( &mut self, - processor_mailbox: &Mailbox

, - ctx: &SourceContext

, + processor_mailbox: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let batch_builder = self.reader.read_batch(ctx.progress()).await?; self.num_bytes_processed += batch_builder.num_bytes; self.num_lines_processed += batch_builder.docs.len() as u64; processor_mailbox - .send_message(batch_builder.build()) + .send_raw_doc_batch(batch_builder.build(), ctx) .await?; if self.reader.is_eof() { - ctx.send_exit_with_success(processor_mailbox).await?; + processor_mailbox.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index aad8136d34b..a176f0d01b0 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -16,7 +16,7 @@ use std::fmt; use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{ActorExitStatus, Mailbox}; +use quickwit_actors::ActorExitStatus; use quickwit_config::VecSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpointDelta}; use quickwit_proto::metastore::SourceType; @@ -25,8 +25,7 @@ use serde_json::Value as JsonValue; use tracing::info; use super::BatchBuilder; -use crate::actors::Processor; -use crate::source::{Source, SourceContext, SourceRuntime, TypedSourceFactory}; +use crate::source::{ProcessorMailbox, Source, SourceContext, SourceRuntime, TypedSourceFactory}; pub struct VecSource { source_id: SourceId, @@ -81,11 +80,11 @@ fn position_from_offset(offset: usize) -> Position { } #[async_trait] -impl Source

for VecSource { +impl Source for VecSource { async fn emit_batches( &mut self, - batch_sink: &Mailbox

, - ctx: &SourceContext

, + batch_sink: &ProcessorMailbox, + ctx: &SourceContext, ) -> Result { let mut batch_builder = BatchBuilder::new(SourceType::Vec); @@ -98,7 +97,7 @@ impl Source

for VecSource { } if batch_builder.docs.is_empty() { info!("reached end of source"); - ctx.send_exit_with_success(batch_sink).await?; + batch_sink.send_exit_with_success(ctx).await?; return Err(ActorExitStatus::Success); } let from_item_idx = self.next_item_idx; @@ -111,7 +110,9 @@ impl Source

for VecSource { position_from_offset(to_item_idx), ) .unwrap(); - ctx.send_message(batch_sink, batch_builder.build()).await?; + batch_sink + .send_raw_doc_batch(batch_builder.build(), ctx) + .await?; Ok(Duration::default()) } @@ -140,8 +141,8 @@ mod tests { use super::*; use crate::actors::DocProcessor; use crate::models::RawDocBatch; - use crate::source::SourceActor; use crate::source::tests::SourceRuntimeBuilder; + use crate::source::{ProcessorMailbox, SourceActor}; #[tokio::test] async fn test_vec_source() -> anyhow::Result<()> { @@ -169,7 +170,7 @@ mod tests { let vec_source = VecSourceFactory::typed_create_source(source_runtime, params).await?; let vec_source_actor = SourceActor { source: Box::new(vec_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; assert_eq!( vec_source_actor.name(), @@ -221,7 +222,7 @@ mod tests { let vec_source = VecSourceFactory::typed_create_source(source_runtime, params).await?; let vec_source_actor = SourceActor { source: Box::new(vec_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_vec_source_mailbox, vec_source_handle) = universe.spawn_builder().spawn(vec_source_actor); diff --git a/quickwit/quickwit-indexing/src/source/void_source.rs b/quickwit/quickwit-indexing/src/source/void_source.rs index 750a18c6fe5..ed108901194 100644 --- a/quickwit/quickwit-indexing/src/source/void_source.rs +++ b/quickwit/quickwit-indexing/src/source/void_source.rs @@ -15,21 +15,20 @@ use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{ActorExitStatus, HEARTBEAT, Mailbox}; +use quickwit_actors::{ActorExitStatus, HEARTBEAT}; use quickwit_config::VoidSourceParams; use serde_json::Value as JsonValue; -use crate::actors::Processor; -use crate::source::{Source, SourceContext, SourceRuntime, TypedSourceFactory}; +use crate::source::{ProcessorMailbox, Source, SourceContext, SourceRuntime, TypedSourceFactory}; pub struct VoidSource; #[async_trait] -impl Source

for VoidSource { +impl Source for VoidSource { async fn emit_batches( &mut self, - _: &Mailbox

, - _: &SourceContext

, + _: &ProcessorMailbox, + _: &SourceContext, ) -> Result { tokio::time::sleep(*HEARTBEAT / 2).await; Ok(Duration::default()) @@ -73,7 +72,7 @@ mod tests { use super::*; use crate::actors::DocProcessor; use crate::source::tests::SourceRuntimeBuilder; - use crate::source::{SourceActor, SourceConfig, quickwit_supported_sources}; + use crate::source::{ProcessorMailbox, SourceActor, SourceConfig, quickwit_supported_sources}; #[tokio::test] async fn test_void_source_loading() { @@ -112,7 +111,7 @@ mod tests { let (doc_processor_mailbox, _) = universe.create_test_mailbox::(); let void_source_actor = SourceActor { source: Box::new(void_source), - processor_mailbox: doc_processor_mailbox, + processor_mailbox: ProcessorMailbox::new(doc_processor_mailbox), }; let (_, void_source_handle) = universe.spawn_builder().spawn(void_source_actor); matches!(void_source_handle.check_health(true), Health::Healthy); diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 9506a587bd4..be11509c3d7 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -18,7 +18,7 @@ use std::time::Duration; use async_trait::async_trait; use quickwit_actors::{ - Actor, ActorContext, ActorExitStatus, ActorHandle, Handler, Mailbox, Supervisor, + Actor, ActorContext, ActorExitStatus, ActorHandle, Handler, Mailbox, QueueCapacity, Supervisor, SupervisorState, }; use quickwit_common::io::IoControls; @@ -31,7 +31,7 @@ use quickwit_indexing::actors::{ PublisherCounters, Uploader, UploaderCounters, UploaderType, }; use quickwit_indexing::merge_policy::merge_policy_from_settings; -use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; +use quickwit_indexing::{IndexingSplitStore, SplitsUpdateMailbox}; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; @@ -162,7 +162,8 @@ impl DeleteTaskPipeline { .deserialize_index_metadata()? .into_index_config(); let publisher = Publisher::new( - PublisherType::MergePublisher, + "MergePublisher", + QueueCapacity::Unbounded, self.metastore.clone(), None, None,