From 7a7207b6640918ffb65f01c585d297e349dbd6a0 Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Mon, 6 Apr 2026 13:38:15 -0400 Subject: [PATCH 01/11] Make MergeSchedulerService optional (#6266) Make MergeSchedulerService optional --- quickwit/quickwit-cli/src/tool.rs | 4 +- .../src/actors/indexing_pipeline.rs | 12 +- .../src/actors/indexing_service.rs | 148 ++++++++++++++---- .../quickwit-indexing/src/actors/publisher.rs | 21 ++- quickwit/quickwit-indexing/src/lib.rs | 5 +- quickwit/quickwit-indexing/src/test_utils.rs | 2 +- quickwit/quickwit-serve/src/lib.rs | 6 +- 7 files changed, 148 insertions(+), 50 deletions(-) diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 0f34a3017e6..04843d7e801 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -456,7 +456,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result< cluster, metastore, None, - merge_scheduler_service_mailbox, + Some(merge_scheduler_service_mailbox), IngesterPool::default(), storage_resolver, EventBroker::default(), @@ -593,7 +593,7 @@ pub async fn merge_cli(args: MergeArgs) -> anyhow::Result<()> { cluster, metastore, None, - merge_scheduler_service, + Some(merge_scheduler_service), IngesterPool::default(), storage_resolver, EventBroker::default(), diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 16daf18102f..04a9ab004b6 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -431,7 +431,7 @@ impl IndexingPipeline { let publisher = Publisher::new( PublisherType::MainPublisher, self.params.metastore.clone(), - Some(self.params.merge_planner_mailbox.clone()), + self.params.merge_planner_mailbox_opt.clone(), Some(source_mailbox.clone()), ); let (publisher_mailbox, publisher_handle) = ctx @@ -854,7 +854,7 @@ pub struct IndexingPipelineParams { // Merge-related parameters pub merge_policy: Arc, pub retention_policy: Option, - pub merge_planner_mailbox: Mailbox, + pub merge_planner_mailbox_opt: Option>, pub max_concurrent_split_uploads_merge: usize, // Source-related parameters @@ -991,7 +991,7 @@ mod tests { max_concurrent_split_uploads_index: 4, max_concurrent_split_uploads_merge: 5, cooperative_indexing_permits: None, - merge_planner_mailbox, + merge_planner_mailbox_opt: Some(merge_planner_mailbox), event_broker: EventBroker::default(), params_fingerprint: 42u64, }; @@ -1115,7 +1115,7 @@ mod tests { max_concurrent_split_uploads_index: 4, max_concurrent_split_uploads_merge: 5, cooperative_indexing_permits: None, - merge_planner_mailbox, + merge_planner_mailbox_opt: Some(merge_planner_mailbox), event_broker: Default::default(), params_fingerprint: 42u64, }; @@ -1216,7 +1216,7 @@ mod tests { max_concurrent_split_uploads_index: 4, max_concurrent_split_uploads_merge: 5, cooperative_indexing_permits: None, - merge_planner_mailbox: merge_planner_mailbox.clone(), + merge_planner_mailbox_opt: Some(merge_planner_mailbox.clone()), event_broker: Default::default(), params_fingerprint: 42u64, }; @@ -1344,7 +1344,7 @@ mod tests { max_concurrent_split_uploads_index: 4, max_concurrent_split_uploads_merge: 5, cooperative_indexing_permits: None, - merge_planner_mailbox, + merge_planner_mailbox_opt: Some(merge_planner_mailbox), params_fingerprint: 42u64, event_broker: Default::default(), }; diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 5d307fb200b..247dc53c41a 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -104,7 +104,7 @@ pub struct IndexingService { cluster: Cluster, metastore: MetastoreServiceClient, ingest_api_service_opt: Option>, - merge_scheduler_service: Mailbox, + merge_scheduler_service_opt: Option>, ingester_pool: IngesterPool, storage_resolver: StorageResolver, indexing_pipelines: HashMap, @@ -138,7 +138,7 @@ impl IndexingService { cluster: Cluster, metastore: MetastoreServiceClient, ingest_api_service_opt: Option>, - merge_scheduler_service: Mailbox, + merge_scheduler_service_opt: Option>, ingester_pool: IngesterPool, storage_resolver: StorageResolver, event_broker: EventBroker, @@ -167,7 +167,7 @@ impl IndexingService { cluster, metastore, ingest_api_service_opt, - merge_scheduler_service, + merge_scheduler_service_opt, ingester_pool, storage_resolver, local_split_store: Arc::new(local_split_store), @@ -297,27 +297,37 @@ impl IndexingService { let doc_mapper = build_doc_mapper(&index_config.doc_mapping, &index_config.search_settings) .map_err(|error| IndexingError::Internal(error.to_string()))?; - let merge_pipeline_id = indexing_pipeline_id.merge_pipeline_id(); - let merge_pipeline_params = MergePipelineParams { - pipeline_id: merge_pipeline_id.clone(), - doc_mapper: doc_mapper.clone(), - indexing_directory: indexing_directory.clone(), - metastore: self.metastore.clone(), - split_store: split_store.clone(), - merge_scheduler_service: self.merge_scheduler_service.clone(), - merge_policy: merge_policy.clone(), - retention_policy: retention_policy.clone(), - merge_io_throughput_limiter_opt: self.merge_io_throughput_limiter_opt.clone(), - max_concurrent_split_uploads: self.max_concurrent_split_uploads, - event_broker: self.event_broker.clone(), + let merge_planner_mailbox_opt = if let Some(merge_scheduler_service) = + self.merge_scheduler_service_opt.clone() + { + let merge_pipeline_id = indexing_pipeline_id.merge_pipeline_id(); + let merge_pipeline_params = MergePipelineParams { + pipeline_id: merge_pipeline_id, + doc_mapper: doc_mapper.clone(), + indexing_directory: indexing_directory.clone(), + metastore: self.metastore.clone(), + split_store: split_store.clone(), + merge_scheduler_service, + merge_policy: merge_policy.clone(), + retention_policy: retention_policy.clone(), + merge_io_throughput_limiter_opt: self.merge_io_throughput_limiter_opt.clone(), + max_concurrent_split_uploads: self.max_concurrent_split_uploads, + event_broker: self.event_broker.clone(), + }; + Some(self.get_or_create_merge_pipeline(merge_pipeline_params, immature_splits_opt, ctx)) + } else { + None }; - let merge_planner_mailbox = - self.get_or_create_merge_pipeline(merge_pipeline_params, immature_splits_opt, ctx)?; // The concurrent uploads budget is split in 2: 1/2 for the indexing pipeline, 1/2 for the - // merge pipeline. - let max_concurrent_split_uploads_index = (self.max_concurrent_split_uploads / 2).max(1); + // merge pipeline. When there is no local merge pipeline, the indexing pipeline gets the + // full budget. + let max_concurrent_split_uploads_index = if self.merge_scheduler_service_opt.is_some() { + (self.max_concurrent_split_uploads / 2).max(1) + } else { + self.max_concurrent_split_uploads + }; let max_concurrent_split_uploads_merge = - (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); + self.max_concurrent_split_uploads - max_concurrent_split_uploads_index; let params_fingerprint = indexing_pipeline_params_fingerprint(&index_config, &source_config); @@ -354,7 +364,7 @@ impl IndexingService { merge_policy, retention_policy, max_concurrent_split_uploads_merge, - merge_planner_mailbox, + merge_planner_mailbox_opt, // Source-related parameters source_config, @@ -428,6 +438,9 @@ impl IndexingService { indexing_pipeline_ids: &[IndexingPipelineId], ctx: &ActorContext, ) -> MetastoreResult>> { + if self.merge_scheduler_service_opt.is_none() { + return Ok(Default::default()); + } let mut index_uids = Vec::new(); for indexing_pipeline_id in indexing_pipeline_ids { @@ -573,12 +586,12 @@ impl IndexingService { merge_pipeline_params: MergePipelineParams, immature_splits_opt: Option>, ctx: &ActorContext, - ) -> Result, IndexingError> { + ) -> Mailbox { if let Some(merge_pipeline_handle) = self .merge_pipeline_handles .get(&merge_pipeline_params.pipeline_id) { - return Ok(merge_pipeline_handle.mailbox.clone()); + return merge_pipeline_handle.mailbox.clone(); } let merge_pipeline_id = merge_pipeline_params.pipeline_id.clone(); let merge_pipeline = @@ -592,7 +605,7 @@ impl IndexingService { self.merge_pipeline_handles .insert(merge_pipeline_id, merge_pipeline_handle); self.counters.num_running_merge_pipelines += 1; - Ok(merge_planner_mailbox) + merge_planner_mailbox } /// For all Ingest V2 pipelines, assigns the set of shards they should be working on. @@ -1053,7 +1066,7 @@ mod tests { cluster, metastore, Some(ingest_api_service), - merge_scheduler_mailbox, + Some(merge_scheduler_mailbox), IngesterPool::default(), storage_resolver.clone(), EventBroker::default(), @@ -1566,7 +1579,7 @@ mod tests { cluster.clone(), metastore.clone(), Some(ingest_api_service), - merge_scheduler_service, + Some(merge_scheduler_service), IngesterPool::default(), storage_resolver.clone(), EventBroker::default(), @@ -1610,6 +1623,85 @@ mod tests { universe.quit().await; } + #[tokio::test] + async fn test_indexing_service_no_merge_pipeline_when_no_merge_scheduler() { + quickwit_common::setup_logging_for_tests(); + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + .await + .unwrap(); + let metastore = metastore_for_test(); + + let index_id = append_random_suffix("test-indexing-service-no-merge"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let source_config = SourceConfig { + source_id: "test-source".to_string(), + num_pipelines: NonZeroUsize::MIN, + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + let create_index_request = + CreateIndexRequest::try_from_index_config(&index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid() + .clone(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + + let temp_dir = tempfile::tempdir().unwrap(); + let data_dir_path = temp_dir.path().to_path_buf(); + let indexer_config = IndexerConfig::for_test().unwrap(); + let num_blocking_threads = 1; + let storage_resolver = StorageResolver::unconfigured(); + let universe = Universe::with_accelerated_time(); + let queues_dir_path = data_dir_path.join(QUEUES_DIR_NAME); + let ingest_api_service = + init_ingest_api(&universe, &queues_dir_path, &IngestApiConfig::default()) + .await + .unwrap(); + let indexing_server = IndexingService::new( + NodeId::from("test-node"), + data_dir_path, + indexer_config, + num_blocking_threads, + cluster.clone(), + metastore.clone(), + Some(ingest_api_service), + None, // No merge scheduler — external merge service handles compaction. + IngesterPool::default(), + storage_resolver.clone(), + EventBroker::default(), + ) + .await + .unwrap(); + let (indexing_server_mailbox, indexing_server_handle) = + universe.spawn_builder().spawn(indexing_server); + + indexing_server_mailbox + .ask_for_res(SpawnPipeline { + index_id: index_id.clone(), + source_config, + pipeline_uid: PipelineUid::default(), + }) + .await + .unwrap(); + + let observation = indexing_server_handle.observe().await; + assert_eq!(observation.num_running_pipelines, 1); + assert_eq!(observation.num_running_merge_pipelines, 0); + assert!(universe.get_one::().is_none()); + + universe.quit().await; + } + #[derive(Debug)] struct FreezePipeline; #[async_trait] @@ -1768,7 +1860,7 @@ mod tests { cluster.clone(), metastore.clone(), Some(ingest_api_service.clone()), - merge_scheduler_service, + Some(merge_scheduler_service), IngesterPool::default(), storage_resolver.clone(), EventBroker::default(), diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index c4ec75b1a8a..4ed7baec043 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -21,7 +21,7 @@ use quickwit_proto::metastore::{ MetastoreService, MetastoreServiceClient, PublishMetricsSplitsRequest, PublishSplitsRequest, }; use serde::Serialize; -use tracing::{info, instrument}; +use tracing::{error, info, instrument}; use crate::actors::{DocProcessor, MergePlanner, ParquetDocProcessor, Processor}; use crate::models::{NewSplits, ParquetSplitsUpdate, SplitsUpdate}; @@ -210,14 +210,19 @@ impl Handler for Publisher { 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 + // When merging is handled locally, notify the merge planner about new + // splits. The mailbox is None when an external merge service is active, + // or when the planner has already shut down (e.g. source reached its end). + if let Some(merge_planner_mailbox) = &self.merge_planner_mailbox_opt { + match ctx .send_message(merge_planner_mailbox, NewSplits { new_splits }) - .await; + .await + { + Ok(_) => {} + Err(error) => { + error!(error=?error, "failed to send new splits to merge planner"); + } + } } if replaced_split_ids.is_empty() { diff --git a/quickwit/quickwit-indexing/src/lib.rs b/quickwit/quickwit-indexing/src/lib.rs index 2c2b28a09d7..8adde285bf3 100644 --- a/quickwit/quickwit-indexing/src/lib.rs +++ b/quickwit/quickwit-indexing/src/lib.rs @@ -69,13 +69,10 @@ pub async fn start_indexing_service( ingester_pool: IngesterPool, storage_resolver: StorageResolver, event_broker: EventBroker, + merge_scheduler_mailbox: Option>, ) -> anyhow::Result> { info!("starting indexer service"); let ingest_api_service_mailbox = universe.get_one::(); - let (merge_scheduler_mailbox, _) = universe.spawn_builder().spawn(MergeSchedulerService::new( - config.indexer_config.merge_concurrency.get(), - )); - // Spawn indexing service. let indexing_service = IndexingService::new( config.node_id.clone(), config.data_dir_path.to_path_buf(), diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 82198c820f4..550e40aa16f 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -123,7 +123,7 @@ impl TestSandbox { cluster, metastore.clone(), Some(ingest_api_service), - merge_scheduler_mailbox, + Some(merge_scheduler_mailbox), IngesterPool::default(), storage_resolver.clone(), EventBroker::default(), diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index cc261cec7a2..4d674d9cb7b 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -77,7 +77,7 @@ use quickwit_config::{ClusterConfig, IngestApiConfig, NodeConfig}; use quickwit_control_plane::control_plane::{ControlPlane, ControlPlaneEventSubscriber}; use quickwit_control_plane::{IndexerNodeInfo, IndexerPool}; use quickwit_index_management::{IndexService as IndexManager, IndexServiceError}; -use quickwit_indexing::actors::IndexingService; +use quickwit_indexing::actors::{IndexingService, MergeSchedulerService}; use quickwit_indexing::models::ShardPositionsService; use quickwit_indexing::start_indexing_service; use quickwit_ingest::{ @@ -540,6 +540,9 @@ pub async fn serve_quickwit( .context("failed to start ingest v1 service")?; let indexing_service_opt = if node_config.is_service_enabled(QuickwitService::Indexer) { + let (merge_scheduler_mailbox, _) = universe.spawn_builder().spawn( + MergeSchedulerService::new(node_config.indexer_config.merge_concurrency.get()), + ); let indexing_service = start_indexing_service( &universe, &node_config, @@ -549,6 +552,7 @@ pub async fn serve_quickwit( ingester_pool.clone(), storage_resolver.clone(), event_broker.clone(), + Some(merge_scheduler_mailbox), ) .await .context("failed to start indexing service")?; From 2862e8740141c45b0fce5fabb7c3910f7b9df17f Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Thu, 9 Apr 2026 14:43:07 -0400 Subject: [PATCH 02/11] Create compaction planner service stub; wire up serving layer (#6269) --- quickwit/Cargo.lock | 9 + quickwit/Cargo.toml | 3 + quickwit/quickwit-compaction/Cargo.toml | 15 + quickwit/quickwit-compaction/src/lib.rs | 17 + .../src/planner/compaction_service.rs | 26 + .../quickwit-compaction/src/planner/mod.rs | 17 + quickwit/quickwit-config/src/service.rs | 3 + .../src/actors/indexing_service.rs | 80 ++ .../quickwit-indexing/src/actors/publisher.rs | 12 +- .../src/tests/basic_tests.rs | 96 +++ quickwit/quickwit-proto/build.rs | 14 + .../protos/quickwit/compaction.proto | 24 + .../codegen/quickwit/quickwit.compaction.rs | 797 ++++++++++++++++++ quickwit/quickwit-proto/src/compaction/mod.rs | 76 ++ quickwit/quickwit-proto/src/lib.rs | 1 + quickwit/quickwit-serve/Cargo.toml | 1 + quickwit/quickwit-serve/src/grpc.rs | 16 + quickwit/quickwit-serve/src/lib.rs | 120 ++- quickwit/quickwit-serve/src/rest.rs | 1 + 19 files changed, 1316 insertions(+), 12 deletions(-) create mode 100644 quickwit/quickwit-compaction/Cargo.toml create mode 100644 quickwit/quickwit-compaction/src/lib.rs create mode 100644 quickwit/quickwit-compaction/src/planner/compaction_service.rs create mode 100644 quickwit/quickwit-compaction/src/planner/mod.rs create mode 100644 quickwit/quickwit-proto/protos/quickwit/compaction.proto create mode 100644 quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs create mode 100644 quickwit/quickwit-proto/src/compaction/mod.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 89eefe2bed7..544b882d332 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7144,6 +7144,14 @@ dependencies = [ "tracing", ] +[[package]] +name = "quickwit-compaction" +version = "0.8.0" +dependencies = [ + "async-trait", + "quickwit-proto", +] + [[package]] name = "quickwit-config" version = "0.8.0" @@ -7808,6 +7816,7 @@ dependencies = [ "quickwit-actors", "quickwit-cluster", "quickwit-common", + "quickwit-compaction", "quickwit-config", "quickwit-control-plane", "quickwit-doc-mapper", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 9242390d898..9d8cf833dc3 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -12,6 +12,7 @@ members = [ "quickwit-control-plane", "quickwit-datetime", "quickwit-directories", + "quickwit-compaction", "quickwit-doc-mapper", "quickwit-index-management", "quickwit-indexing", @@ -52,6 +53,7 @@ default-members = [ "quickwit-control-plane", "quickwit-datetime", "quickwit-directories", + "quickwit-compaction", "quickwit-doc-mapper", "quickwit-index-management", "quickwit-indexing", @@ -352,6 +354,7 @@ quickwit-control-plane = { path = "quickwit-control-plane" } quickwit-datetime = { path = "quickwit-datetime" } quickwit-directories = { path = "quickwit-directories" } quickwit-doc-mapper = { path = "quickwit-doc-mapper" } +quickwit-compaction = { path = "quickwit-compaction" } quickwit-index-management = { path = "quickwit-index-management" } quickwit-indexing = { path = "quickwit-indexing" } quickwit-ingest = { path = "quickwit-ingest" } diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml new file mode 100644 index 00000000000..07925537dce --- /dev/null +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -0,0 +1,15 @@ +[package] +name = "quickwit-compaction" +description = "Merge planner and merge worker services for split compaction" + +version.workspace = true +edition.workspace = true +homepage.workspace = true +documentation.workspace = true +repository.workspace = true +authors.workspace = true +license.workspace = true + +[dependencies] +async-trait = { workspace = true } +quickwit-proto = { workspace = true } \ No newline at end of file diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs new file mode 100644 index 00000000000..d59933e227e --- /dev/null +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -0,0 +1,17 @@ +// 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. + +#![deny(clippy::disallowed_methods)] + +pub mod planner; diff --git a/quickwit/quickwit-compaction/src/planner/compaction_service.rs b/quickwit/quickwit-compaction/src/planner/compaction_service.rs new file mode 100644 index 00000000000..0e3104a2129 --- /dev/null +++ b/quickwit/quickwit-compaction/src/planner/compaction_service.rs @@ -0,0 +1,26 @@ +// 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. + +use async_trait::async_trait; +use quickwit_proto::compaction::{CompactionResult, CompactionService, PingRequest, PingResponse}; + +#[derive(Debug, Clone)] +pub struct StubCompactionService; + +#[async_trait] +impl CompactionService for StubCompactionService { + async fn ping(&self, _request: PingRequest) -> CompactionResult { + Ok(PingResponse {}) + } +} diff --git a/quickwit/quickwit-compaction/src/planner/mod.rs b/quickwit/quickwit-compaction/src/planner/mod.rs new file mode 100644 index 00000000000..3b531a439f2 --- /dev/null +++ b/quickwit/quickwit-compaction/src/planner/mod.rs @@ -0,0 +1,17 @@ +// 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. + +mod compaction_service; + +pub use compaction_service::StubCompactionService; diff --git a/quickwit/quickwit-config/src/service.rs b/quickwit/quickwit-config/src/service.rs index d323331b510..0382231330d 100644 --- a/quickwit/quickwit-config/src/service.rs +++ b/quickwit/quickwit-config/src/service.rs @@ -29,6 +29,7 @@ pub enum QuickwitService { Searcher, Janitor, Metastore, + Compactor, } #[allow(clippy::from_over_into)] @@ -46,6 +47,7 @@ impl QuickwitService { QuickwitService::Searcher => "searcher", QuickwitService::Janitor => "janitor", QuickwitService::Metastore => "metastore", + QuickwitService::Compactor => "compactor", } } @@ -70,6 +72,7 @@ impl FromStr for QuickwitService { "searcher" => Ok(QuickwitService::Searcher), "janitor" => Ok(QuickwitService::Janitor), "metastore" => Ok(QuickwitService::Metastore), + "compactor" => Ok(QuickwitService::Compactor), _ => { bail!( "failed to parse service `{service_str}`. supported services are: `{}`", diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 247dc53c41a..5442a98589e 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -1702,6 +1702,86 @@ mod tests { universe.quit().await; } + #[tokio::test] + async fn test_indexing_service_spawns_merge_pipeline_with_merge_scheduler() { + quickwit_common::setup_logging_for_tests(); + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + .await + .unwrap(); + let metastore = metastore_for_test(); + + let index_id = append_random_suffix("test-indexing-service-with-merge"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let source_config = SourceConfig { + source_id: "test-source".to_string(), + num_pipelines: NonZeroUsize::MIN, + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + let create_index_request = + CreateIndexRequest::try_from_index_config(&index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid() + .clone(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + + let temp_dir = tempfile::tempdir().unwrap(); + let data_dir_path = temp_dir.path().to_path_buf(); + let indexer_config = IndexerConfig::for_test().unwrap(); + let num_blocking_threads = 1; + let storage_resolver = StorageResolver::unconfigured(); + let universe = Universe::with_accelerated_time(); + let queues_dir_path = data_dir_path.join(QUEUES_DIR_NAME); + let ingest_api_service = + init_ingest_api(&universe, &queues_dir_path, &IngestApiConfig::default()) + .await + .unwrap(); + let merge_scheduler_mailbox: Mailbox = universe.get_or_spawn_one(); + let indexing_server = IndexingService::new( + NodeId::from("test-node"), + data_dir_path, + indexer_config, + num_blocking_threads, + cluster.clone(), + metastore.clone(), + Some(ingest_api_service), + Some(merge_scheduler_mailbox), + IngesterPool::default(), + storage_resolver.clone(), + EventBroker::default(), + ) + .await + .unwrap(); + let (indexing_server_mailbox, indexing_server_handle) = + universe.spawn_builder().spawn(indexing_server); + + indexing_server_mailbox + .ask_for_res(SpawnPipeline { + index_id: index_id.clone(), + source_config, + pipeline_uid: PipelineUid::default(), + }) + .await + .unwrap(); + + let observation = indexing_server_handle.observe().await; + assert_eq!(observation.num_running_pipelines, 1); + assert_eq!(observation.num_running_merge_pipelines, 1); + assert!(universe.get_one::().is_some()); + + universe.quit().await; + } + #[derive(Debug)] struct FreezePipeline; #[async_trait] diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 4ed7baec043..e0b4b103a4b 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -213,16 +213,12 @@ impl Handler for Publisher { // When merging is handled locally, notify the merge planner about new // splits. The mailbox is None when an external merge service is active, // or when the planner has already shut down (e.g. source reached its end). - if let Some(merge_planner_mailbox) = &self.merge_planner_mailbox_opt { - match ctx + if let Some(merge_planner_mailbox) = &self.merge_planner_mailbox_opt + && let Err(error) = ctx .send_message(merge_planner_mailbox, NewSplits { new_splits }) .await - { - Ok(_) => {} - Err(error) => { - error!(error=?error, "failed to send new splits to merge planner"); - } - } + { + error!(error=?error, "failed to send new splits to merge planner"); } if replaced_split_ids.is_empty() { diff --git a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs index 91c6b0ebeb1..2715e310a97 100644 --- a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs @@ -162,3 +162,99 @@ async fn test_multi_nodes_cluster() { sandbox.shutdown().await.unwrap(); } + +#[tokio::test] +async fn test_no_merge_pipelines_when_compaction_service_enabled() { + quickwit_common::setup_logging_for_tests(); + unsafe { std::env::set_var("QW_ENABLE_COMPACTION_SERVICE", "true") }; + + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; + + sandbox + .rest_client(QuickwitService::Indexer) + .indexes() + .create( + r#" + version: 0.8 + index_id: test-no-merge-pipelines + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 1 + "#, + quickwit_config::ConfigFormat::Yaml, + false, + ) + .await + .unwrap(); + + sandbox.wait_for_indexing_pipelines(1).await.unwrap(); + + let stats = sandbox + .rest_client(QuickwitService::Indexer) + .node_stats() + .indexing() + .await + .unwrap(); + assert_eq!(stats.num_running_merge_pipelines, 0); + + unsafe { std::env::remove_var("QW_ENABLE_COMPACTION_SERVICE") }; + sandbox.shutdown().await.unwrap(); +} + +#[tokio::test] +async fn test_merge_pipelines_present_without_compaction_service() { + quickwit_common::setup_logging_for_tests(); + unsafe { std::env::set_var("QW_ENABLE_COMPACTION_SERVICE", "false") }; + + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; + + sandbox + .rest_client(QuickwitService::Indexer) + .indexes() + .create( + r#" + version: 0.8 + index_id: test-with-merge-pipelines + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 1 + "#, + quickwit_config::ConfigFormat::Yaml, + false, + ) + .await + .unwrap(); + + sandbox.wait_for_indexing_pipelines(1).await.unwrap(); + + let stats = sandbox + .rest_client(QuickwitService::Indexer) + .node_stats() + .indexing() + .await + .unwrap(); + assert!(stats.num_running_merge_pipelines > 0); + + unsafe { std::env::remove_var("QW_ENABLE_COMPACTION_SERVICE") }; + sandbox.shutdown().await.unwrap(); +} diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index 569d9b5315b..a83e8723bd2 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -35,6 +35,20 @@ fn main() -> Result<(), Box> { .run() .unwrap(); + // Compaction service. + let mut prost_config = prost_build::Config::default(); + prost_config.file_descriptor_set_path("src/codegen/quickwit/compaction_descriptor.bin"); + + Codegen::builder() + .with_prost_config(prost_config) + .with_protos(&["protos/quickwit/compaction.proto"]) + .with_output_dir("src/codegen/quickwit") + .with_result_type_path("crate::compaction::CompactionResult") + .with_error_type_path("crate::compaction::CompactionError") + .generate_rpc_name_impls() + .run() + .unwrap(); + // Control plane. let mut prost_config = prost_build::Config::default(); prost_config.file_descriptor_set_path("src/codegen/quickwit/control_plane_descriptor.bin"); diff --git a/quickwit/quickwit-proto/protos/quickwit/compaction.proto b/quickwit/quickwit-proto/protos/quickwit/compaction.proto new file mode 100644 index 00000000000..82979c3b4ee --- /dev/null +++ b/quickwit/quickwit-proto/protos/quickwit/compaction.proto @@ -0,0 +1,24 @@ +// 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. + +syntax = "proto3"; + +package quickwit.compaction; + +service CompactionService { + rpc Ping(PingRequest) returns (PingResponse); +} + +message PingRequest {} +message PingResponse {} \ No newline at end of file diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs new file mode 100644 index 00000000000..eec6be161ca --- /dev/null +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs @@ -0,0 +1,797 @@ +// This file is @generated by prost-build. +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct PingRequest {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct PingResponse {} +/// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; +use tower::{Layer, Service, ServiceExt}; +use quickwit_common::tower::RpcName; +impl RpcName for PingRequest { + fn rpc_name() -> &'static str { + "ping" + } +} +#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] +#[async_trait::async_trait] +pub trait CompactionService: std::fmt::Debug + Send + Sync + 'static { + async fn ping( + &self, + request: PingRequest, + ) -> crate::compaction::CompactionResult; +} +#[derive(Debug, Clone)] +pub struct CompactionServiceClient { + inner: InnerCompactionServiceClient, +} +#[derive(Debug, Clone)] +struct InnerCompactionServiceClient(std::sync::Arc); +impl CompactionServiceClient { + pub fn new(instance: T) -> Self + where + T: CompactionService, + { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockCompactionService > (), + "`MockCompactionService` must be wrapped in a `MockCompactionServiceWrapper`: use `CompactionServiceClient::from_mock(mock)` to instantiate the client" + ); + Self { + inner: InnerCompactionServiceClient(std::sync::Arc::new(instance)), + } + } + pub fn as_grpc_service( + &self, + max_message_size: bytesize::ByteSize, + ) -> compaction_service_grpc_server::CompactionServiceGrpcServer< + CompactionServiceGrpcServerAdapter, + > { + let adapter = CompactionServiceGrpcServerAdapter::new(self.clone()); + compaction_service_grpc_server::CompactionServiceGrpcServer::new(adapter) + .accept_compressed(tonic::codec::CompressionEncoding::Gzip) + .accept_compressed(tonic::codec::CompressionEncoding::Zstd) + .send_compressed(tonic::codec::CompressionEncoding::Gzip) + .send_compressed(tonic::codec::CompressionEncoding::Zstd) + .max_decoding_message_size(max_message_size.0 as usize) + .max_encoding_message_size(max_message_size.0 as usize) + } + pub fn from_channel( + addr: std::net::SocketAddr, + channel: tonic::transport::Channel, + max_message_size: bytesize::ByteSize, + compression_encoding_opt: Option, + ) -> Self { + let (_, connection_keys_watcher) = tokio::sync::watch::channel( + std::collections::HashSet::from_iter([addr]), + ); + let mut client = compaction_service_grpc_client::CompactionServiceGrpcClient::new( + channel, + ) + .max_decoding_message_size(max_message_size.0 as usize) + .max_encoding_message_size(max_message_size.0 as usize); + if let Some(compression_encoding) = compression_encoding_opt { + client = client + .accept_compressed(compression_encoding) + .send_compressed(compression_encoding); + } + let adapter = CompactionServiceGrpcClientAdapter::new( + client, + connection_keys_watcher, + ); + Self::new(adapter) + } + pub fn from_balance_channel( + balance_channel: quickwit_common::tower::BalanceChannel, + max_message_size: bytesize::ByteSize, + compression_encoding_opt: Option, + ) -> CompactionServiceClient { + let connection_keys_watcher = balance_channel.connection_keys_watcher(); + let mut client = compaction_service_grpc_client::CompactionServiceGrpcClient::new( + balance_channel, + ) + .max_decoding_message_size(max_message_size.0 as usize) + .max_encoding_message_size(max_message_size.0 as usize); + if let Some(compression_encoding) = compression_encoding_opt { + client = client + .accept_compressed(compression_encoding) + .send_compressed(compression_encoding); + } + let adapter = CompactionServiceGrpcClientAdapter::new( + client, + connection_keys_watcher, + ); + Self::new(adapter) + } + pub fn from_mailbox(mailbox: quickwit_actors::Mailbox) -> Self + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + CompactionServiceMailbox: CompactionService, + { + CompactionServiceClient::new(CompactionServiceMailbox::new(mailbox)) + } + pub fn tower() -> CompactionServiceTowerLayerStack { + CompactionServiceTowerLayerStack::default() + } + #[cfg(any(test, feature = "testsuite"))] + pub fn from_mock(mock: MockCompactionService) -> Self { + let mock_wrapper = mock_compaction_service::MockCompactionServiceWrapper { + inner: tokio::sync::Mutex::new(mock), + }; + Self::new(mock_wrapper) + } + #[cfg(any(test, feature = "testsuite"))] + pub fn mocked() -> Self { + Self::from_mock(MockCompactionService::new()) + } +} +#[async_trait::async_trait] +impl CompactionService for CompactionServiceClient { + async fn ping( + &self, + request: PingRequest, + ) -> crate::compaction::CompactionResult { + self.inner.0.ping(request).await + } +} +#[cfg(any(test, feature = "testsuite"))] +pub mod mock_compaction_service { + use super::*; + #[derive(Debug)] + pub struct MockCompactionServiceWrapper { + pub(super) inner: tokio::sync::Mutex, + } + #[async_trait::async_trait] + impl CompactionService for MockCompactionServiceWrapper { + async fn ping( + &self, + request: super::PingRequest, + ) -> crate::compaction::CompactionResult { + self.inner.lock().await.ping(request).await + } + } +} +pub type BoxFuture = std::pin::Pin< + Box> + Send + 'static>, +>; +impl tower::Service for InnerCompactionServiceClient { + type Response = PingResponse; + type Error = crate::compaction::CompactionError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: PingRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.ping(request).await }; + Box::pin(fut) + } +} +/// A tower service stack is a set of tower services. +#[derive(Debug)] +struct CompactionServiceTowerServiceStack { + #[allow(dead_code)] + inner: InnerCompactionServiceClient, + ping_svc: quickwit_common::tower::BoxService< + PingRequest, + PingResponse, + crate::compaction::CompactionError, + >, +} +#[async_trait::async_trait] +impl CompactionService for CompactionServiceTowerServiceStack { + async fn ping( + &self, + request: PingRequest, + ) -> crate::compaction::CompactionResult { + self.ping_svc.clone().ready().await?.call(request).await + } +} +type PingLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + PingRequest, + PingResponse, + crate::compaction::CompactionError, + >, + PingRequest, + PingResponse, + crate::compaction::CompactionError, +>; +#[derive(Debug, Default)] +pub struct CompactionServiceTowerLayerStack { + ping_layers: Vec, +} +impl CompactionServiceTowerLayerStack { + pub fn stack_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + PingRequest, + PingResponse, + crate::compaction::CompactionError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::compaction::CompactionError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, + { + self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self + } + pub fn stack_ping_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + PingRequest, + PingResponse, + crate::compaction::CompactionError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::compaction::CompactionError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn build(self, instance: T) -> CompactionServiceClient + where + T: CompactionService, + { + let inner_client = InnerCompactionServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) + } + pub fn build_from_channel( + self, + addr: std::net::SocketAddr, + channel: tonic::transport::Channel, + max_message_size: bytesize::ByteSize, + compression_encoding_opt: Option, + ) -> CompactionServiceClient { + let client = CompactionServiceClient::from_channel( + addr, + channel, + max_message_size, + compression_encoding_opt, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) + } + pub fn build_from_balance_channel( + self, + balance_channel: quickwit_common::tower::BalanceChannel, + max_message_size: bytesize::ByteSize, + compression_encoding_opt: Option, + ) -> CompactionServiceClient { + let client = CompactionServiceClient::from_balance_channel( + balance_channel, + max_message_size, + compression_encoding_opt, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) + } + pub fn build_from_mailbox( + self, + mailbox: quickwit_actors::Mailbox, + ) -> CompactionServiceClient + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + CompactionServiceMailbox: CompactionService, + { + let inner_client = InnerCompactionServiceClient( + std::sync::Arc::new(CompactionServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) + } + #[cfg(any(test, feature = "testsuite"))] + pub fn build_from_mock( + self, + mock: MockCompactionService, + ) -> CompactionServiceClient { + let client = CompactionServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) + } + fn build_from_inner_client( + self, + inner_client: InnerCompactionServiceClient, + ) -> CompactionServiceClient { + let ping_svc = self + .ping_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let tower_svc_stack = CompactionServiceTowerServiceStack { + inner: inner_client, + ping_svc, + }; + CompactionServiceClient::new(tower_svc_stack) + } +} +#[derive(Debug, Clone)] +struct MailboxAdapter { + inner: quickwit_actors::Mailbox, + phantom: std::marker::PhantomData, +} +impl std::ops::Deref for MailboxAdapter +where + A: quickwit_actors::Actor, +{ + type Target = quickwit_actors::Mailbox; + fn deref(&self) -> &Self::Target { + &self.inner + } +} +#[derive(Debug)] +pub struct CompactionServiceMailbox { + inner: MailboxAdapter, +} +impl CompactionServiceMailbox { + pub fn new(instance: quickwit_actors::Mailbox) -> Self { + let inner = MailboxAdapter { + inner: instance, + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl Clone for CompactionServiceMailbox { + fn clone(&self) -> Self { + let inner = MailboxAdapter { + inner: self.inner.clone(), + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl tower::Service for CompactionServiceMailbox +where + A: quickwit_actors::Actor + + quickwit_actors::DeferableReplyHandler> + Send + + 'static, + M: std::fmt::Debug + Send + 'static, + T: Send + 'static, + E: std::fmt::Debug + Send + 'static, + crate::compaction::CompactionError: From>, +{ + type Response = T; + type Error = crate::compaction::CompactionError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + //! This does not work with balance middlewares such as `tower::balance::pool::Pool` because + //! this always returns `Poll::Ready`. The fix is to acquire a permit from the + //! mailbox in `poll_ready` and consume it in `call`. + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, message: M) -> Self::Future { + let mailbox = self.inner.clone(); + let fut = async move { + mailbox.ask_for_res(message).await.map_err(|error| error.into()) + }; + Box::pin(fut) + } +} +#[async_trait::async_trait] +impl CompactionService for CompactionServiceMailbox +where + A: quickwit_actors::Actor + std::fmt::Debug, + CompactionServiceMailbox< + A, + >: tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::compaction::CompactionError, + Future = BoxFuture, + >, +{ + async fn ping( + &self, + request: PingRequest, + ) -> crate::compaction::CompactionResult { + self.clone().call(request).await + } +} +#[derive(Debug, Clone)] +pub struct CompactionServiceGrpcClientAdapter { + inner: T, + #[allow(dead_code)] + connection_addrs_rx: tokio::sync::watch::Receiver< + std::collections::HashSet, + >, +} +impl CompactionServiceGrpcClientAdapter { + pub fn new( + instance: T, + connection_addrs_rx: tokio::sync::watch::Receiver< + std::collections::HashSet, + >, + ) -> Self { + Self { + inner: instance, + connection_addrs_rx, + } + } +} +#[async_trait::async_trait] +impl CompactionService +for CompactionServiceGrpcClientAdapter< + compaction_service_grpc_client::CompactionServiceGrpcClient, +> +where + T: tonic::client::GrpcService + std::fmt::Debug + Clone + Send + + Sync + 'static, + T::ResponseBody: tonic::codegen::Body + Send + 'static, + ::Error: Into + + Send, + T::Future: Send, +{ + async fn ping( + &self, + request: PingRequest, + ) -> crate::compaction::CompactionResult { + self.inner + .clone() + .ping(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + PingRequest::rpc_name(), + )) + } +} +#[derive(Debug)] +pub struct CompactionServiceGrpcServerAdapter { + inner: InnerCompactionServiceClient, +} +impl CompactionServiceGrpcServerAdapter { + pub fn new(instance: T) -> Self + where + T: CompactionService, + { + Self { + inner: InnerCompactionServiceClient(std::sync::Arc::new(instance)), + } + } +} +#[async_trait::async_trait] +impl compaction_service_grpc_server::CompactionServiceGrpc +for CompactionServiceGrpcServerAdapter { + async fn ping( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .ping(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } +} +/// Generated client implementations. +pub mod compaction_service_grpc_client { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value, + )] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct CompactionServiceGrpcClient { + inner: tonic::client::Grpc, + } + impl CompactionServiceGrpcClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl CompactionServiceGrpcClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + std::marker::Send + 'static, + ::Error: Into + std::marker::Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> CompactionServiceGrpcClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + std::marker::Send + std::marker::Sync, + { + CompactionServiceGrpcClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + pub async fn ping( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.compaction.CompactionService/Ping", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new("quickwit.compaction.CompactionService", "Ping"), + ); + self.inner.unary(req, path, codec).await + } + } +} +/// Generated server implementations. +pub mod compaction_service_grpc_server { + #![allow( + unused_variables, + dead_code, + missing_docs, + clippy::wildcard_imports, + clippy::let_unit_value, + )] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with CompactionServiceGrpcServer. + #[async_trait] + pub trait CompactionServiceGrpc: std::marker::Send + std::marker::Sync + 'static { + async fn ping( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + } + #[derive(Debug)] + pub struct CompactionServiceGrpcServer { + inner: Arc, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + impl CompactionServiceGrpcServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> + for CompactionServiceGrpcServer + where + T: CompactionServiceGrpc, + B: Body + std::marker::Send + 'static, + B::Error: Into + std::marker::Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + match req.uri().path() { + "/quickwit.compaction.CompactionService/Ping" => { + #[allow(non_camel_case_types)] + struct PingSvc(pub Arc); + impl< + T: CompactionServiceGrpc, + > tonic::server::UnaryService for PingSvc { + type Response = super::PingResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::ping(&inner, request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = PingSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + _ => { + Box::pin(async move { + let mut response = http::Response::new( + tonic::body::Body::default(), + ); + let headers = response.headers_mut(); + headers + .insert( + tonic::Status::GRPC_STATUS, + (tonic::Code::Unimplemented as i32).into(), + ); + headers + .insert( + http::header::CONTENT_TYPE, + tonic::metadata::GRPC_CONTENT_TYPE, + ); + Ok(response) + }) + } + } + } + } + impl Clone for CompactionServiceGrpcServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + /// Generated gRPC service name + pub const SERVICE_NAME: &str = "quickwit.compaction.CompactionService"; + impl tonic::server::NamedService for CompactionServiceGrpcServer { + const NAME: &'static str = SERVICE_NAME; + } +} diff --git a/quickwit/quickwit-proto/src/compaction/mod.rs b/quickwit/quickwit-proto/src/compaction/mod.rs new file mode 100644 index 00000000000..1e27855add3 --- /dev/null +++ b/quickwit/quickwit-proto/src/compaction/mod.rs @@ -0,0 +1,76 @@ +// 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. + +use quickwit_common::rate_limited_error; +use quickwit_common::tower::MakeLoadShedError; +use serde::{Deserialize, Serialize}; + +use crate::GrpcServiceError; +use crate::error::{ServiceError, ServiceErrorCode}; + +include!("../codegen/quickwit/quickwit.compaction.rs"); + +pub const COMPACTION_FILE_DESCRIPTOR_SET: &[u8] = + include_bytes!("../codegen/quickwit/compaction_descriptor.bin"); + +pub type CompactionResult = std::result::Result; + +#[derive(Debug, thiserror::Error, Eq, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "snake_case")] +pub enum CompactionError { + #[error("internal error: {0}")] + Internal(String), + #[error("request timed out: {0}")] + Timeout(String), + #[error("too many requests")] + TooManyRequests, + #[error("service unavailable: {0}")] + Unavailable(String), +} + +impl ServiceError for CompactionError { + fn error_code(&self) -> ServiceErrorCode { + match self { + Self::Internal(err_msg) => { + rate_limited_error!(limit_per_min = 6, "compaction error: {err_msg}"); + ServiceErrorCode::Internal + } + Self::Timeout(_) => ServiceErrorCode::Timeout, + Self::TooManyRequests => ServiceErrorCode::TooManyRequests, + Self::Unavailable(_) => ServiceErrorCode::Unavailable, + } + } +} + +// Required by the codegen tower layers. All four constructors are mandatory. +impl GrpcServiceError for CompactionError { + fn new_internal(message: String) -> Self { + Self::Internal(message) + } + fn new_timeout(message: String) -> Self { + Self::Timeout(message) + } + fn new_too_many_requests() -> Self { + Self::TooManyRequests + } + fn new_unavailable(message: String) -> Self { + Self::Unavailable(message) + } +} + +impl MakeLoadShedError for CompactionError { + fn make_load_shed_error() -> Self { + CompactionError::TooManyRequests + } +} diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index dbe850b55b7..da39dc11711 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -27,6 +27,7 @@ use tracing::Span; use tracing_opentelemetry::OpenTelemetrySpanExt; pub mod cluster; +pub mod compaction; pub mod control_plane; pub use bytes; pub use tonic; diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 2721aa719f3..7b5aa303f26 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -65,6 +65,7 @@ quickwit-control-plane = { workspace = true } quickwit-doc-mapper = { workspace = true } quickwit-index-management = { workspace = true } quickwit-indexing = { workspace = true } +quickwit-compaction = { workspace = true } quickwit-ingest = { workspace = true } quickwit-jaeger = { workspace = true } quickwit-janitor = { workspace = true } diff --git a/quickwit/quickwit-serve/src/grpc.rs b/quickwit/quickwit-serve/src/grpc.rs index 698c9e07d71..382b026a6a8 100644 --- a/quickwit/quickwit-serve/src/grpc.rs +++ b/quickwit/quickwit-serve/src/grpc.rs @@ -141,6 +141,21 @@ pub(crate) async fn start_grpc_server( None }; + // Mount gRPC compaction service if this node is a janitor with the compaction service enabled. + let compaction_grpc_service = if services + .node_config + .is_service_enabled(QuickwitService::Janitor) + { + if let Some(compaction_service) = &services.compaction_service_client_opt { + enabled_grpc_services.insert("compaction"); + file_descriptor_sets.push(quickwit_proto::compaction::COMPACTION_FILE_DESCRIPTOR_SET); + Some(compaction_service.as_grpc_service(grpc_config.max_message_size)) + } else { + None + } + } else { + None + }; // Mount gRPC control plane service if `QuickwitService::ControlPlane` is enabled on node. let control_plane_grpc_service = if services .node_config @@ -238,6 +253,7 @@ pub(crate) async fn start_grpc_server( .add_service(developer_grpc_service) .add_service(health_service) .add_service(reflection_service) + .add_optional_service(compaction_grpc_service) .add_optional_service(control_plane_grpc_service) .add_optional_service(indexing_grpc_service) .add_optional_service(ingest_api_grpc_service) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 4d674d9cb7b..e288161393e 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -72,6 +72,7 @@ use quickwit_common::tower::{ }; use quickwit_common::uri::Uri; use quickwit_common::{get_bool_from_env, spawn_named_task}; +use quickwit_compaction::planner::StubCompactionService; use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IngestApiConfig, NodeConfig}; use quickwit_control_plane::control_plane::{ControlPlane, ControlPlaneEventSubscriber}; @@ -93,6 +94,7 @@ use quickwit_metastore::{ ControlPlaneMetastore, ListIndexesMetadataResponseExt, MetastoreResolver, }; use quickwit_opentelemetry::otlp::{OtlpGrpcLogsService, OtlpGrpcTracesService}; +use quickwit_proto::compaction::CompactionServiceClient; use quickwit_proto::control_plane::ControlPlaneServiceClient; use quickwit_proto::indexing::{IndexingServiceClient, ShardPositionsUpdate}; use quickwit_proto::ingest::ingester::{ @@ -137,9 +139,16 @@ const READINESS_REPORTING_INTERVAL: Duration = if cfg!(any(test, feature = "test Duration::from_secs(10) }; +const COMPACTION_SERVICE_DISCOVERY_TIMEOUT: Duration = if cfg!(any(test, feature = "testsuite")) { + Duration::from_millis(100) +} else { + Duration::from_secs(300) +}; + const METASTORE_CLIENT_MAX_CONCURRENCY_ENV_KEY: &str = "QW_METASTORE_CLIENT_MAX_CONCURRENCY"; const DEFAULT_METASTORE_CLIENT_MAX_CONCURRENCY: usize = 6; const DISABLE_DELETE_TASK_SERVICE_ENV_KEY: &str = "QW_DISABLE_DELETE_TASK_SERVICE"; +const ENABLE_COMPACTION_SERVICE_ENV_KEY: &str = "QW_ENABLE_COMPACTION_SERVICE"; pub type EnvFilterReloadFn = Arc anyhow::Result<()> + Send + Sync>; @@ -195,6 +204,7 @@ struct QuickwitServices { pub ingest_router_service: IngestRouterServiceClient, ingester_opt: Option, + pub compaction_service_client_opt: Option, pub janitor_service_opt: Option>, pub jaeger_service_opt: Option, pub otlp_logs_service_opt: Option, @@ -261,6 +271,56 @@ async fn balance_channel_for_service( BalanceChannel::from_stream(service_change_stream) } +/// Builds a `CompactionServiceClient` if the compaction service is available. +/// +/// On janitor nodes with `QW_ENABLE_COMPACTION_SERVICE=true`, wraps a local stub. +/// On non-janitor nodes with the flag set, waits up to 10s for a remote janitor +/// exposing the gRPC endpoint and logs an error if none is found. +async fn get_compaction_service_client_if_needed( + node_config: &NodeConfig, + cluster: &Cluster, +) -> anyhow::Result, anyhow::Error> { + if !get_bool_from_env(ENABLE_COMPACTION_SERVICE_ENV_KEY, false) { + return Ok(None); + } + // Only janitor nodes (which host the planner) and indexer nodes (which need + // to know whether to spawn local merge pipelines) care about this service. + if !node_config.is_service_enabled(QuickwitService::Indexer) { + return Ok(None); + } + if node_config.is_service_enabled(QuickwitService::Janitor) + && node_config.is_service_enabled(QuickwitService::Indexer) + { + info!("compaction service enabled on this node"); + return Ok(Some(CompactionServiceClient::new(StubCompactionService))); + } + let balance_channel = balance_channel_for_service(cluster, QuickwitService::Janitor).await; + let found = balance_channel + .wait_for(COMPACTION_SERVICE_DISCOVERY_TIMEOUT, |connections| { + !connections.is_empty() + }) + .await; + if !found { + bail!("compaction service is enabled but no janitor node was found in the cluster") + } + info!("remote compaction service detected on janitor node"); + Ok(Some(CompactionServiceClient::from_balance_channel( + balance_channel, + node_config.grpc_config.max_message_size, + None, + ))) +} + +fn spawn_merge_scheduler_service( + universe: &Universe, + node_config: &NodeConfig, +) -> Mailbox { + let (mailbox, _) = universe.spawn_builder().spawn(MergeSchedulerService::new( + node_config.indexer_config.merge_concurrency.get(), + )); + mailbox +} + async fn start_ingest_client_if_needed( node_config: &NodeConfig, universe: &Universe, @@ -539,10 +599,17 @@ pub async fn serve_quickwit( .await .context("failed to start ingest v1 service")?; + let compaction_service_client_opt = + get_compaction_service_client_if_needed(&node_config, &cluster) + .await + .context("failed to initialize compaction service client")?; + let indexing_service_opt = if node_config.is_service_enabled(QuickwitService::Indexer) { - let (merge_scheduler_mailbox, _) = universe.spawn_builder().spawn( - MergeSchedulerService::new(node_config.indexer_config.merge_concurrency.get()), - ); + let merge_scheduler_mailbox_opt = if compaction_service_client_opt.is_none() { + Some(spawn_merge_scheduler_service(&universe, &node_config)) + } else { + None + }; let indexing_service = start_indexing_service( &universe, &node_config, @@ -552,7 +619,7 @@ pub async fn serve_quickwit( ingester_pool.clone(), storage_resolver.clone(), event_broker.clone(), - Some(merge_scheduler_mailbox), + merge_scheduler_mailbox_opt, ) .await .context("failed to start indexing service")?; @@ -766,6 +833,7 @@ pub async fn serve_quickwit( ingest_router_service, ingest_service, ingester_opt: ingester_opt.clone(), + compaction_service_client_opt, janitor_service_opt, jaeger_service_opt, otlp_logs_service_opt, @@ -1814,4 +1882,48 @@ mod tests { assert!(ingester_pool.is_empty()); } + + #[tokio::test] + async fn test_compaction_service_on_janitor_node() { + let transport = ChannelTransport::default(); + let cluster = + create_cluster_for_test(Vec::new(), &["janitor", "indexer"], &transport, true) + .await + .unwrap(); + + // Without the env var, no compaction service. + let mut node_config = NodeConfig::for_test(); + node_config.enabled_services = + HashSet::from([QuickwitService::Janitor, QuickwitService::Indexer]); + let result = get_compaction_service_client_if_needed(&node_config, &cluster) + .await + .unwrap(); + assert!(result.is_none()); + + // With the env var, compaction service client is returned. + unsafe { std::env::set_var(ENABLE_COMPACTION_SERVICE_ENV_KEY, "true") }; + let result = get_compaction_service_client_if_needed(&node_config, &cluster) + .await + .unwrap(); + assert!(result.is_some()); + + unsafe { std::env::remove_var(ENABLE_COMPACTION_SERVICE_ENV_KEY) }; + } + + #[tokio::test] + async fn test_compaction_service_returns_error_when_no_janitor() { + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, false) + .await + .unwrap(); + + unsafe { std::env::set_var(ENABLE_COMPACTION_SERVICE_ENV_KEY, "true") }; + + let mut node_config = NodeConfig::for_test(); + node_config.enabled_services = HashSet::from([QuickwitService::Indexer]); + let result = get_compaction_service_client_if_needed(&node_config, &cluster).await; + assert!(result.is_err()); + + unsafe { std::env::remove_var(ENABLE_COMPACTION_SERVICE_ENV_KEY) }; + } } diff --git a/quickwit/quickwit-serve/src/rest.rs b/quickwit/quickwit-serve/src/rest.rs index 3f193783b04..c52d790167d 100644 --- a/quickwit/quickwit-serve/src/rest.rs +++ b/quickwit/quickwit-serve/src/rest.rs @@ -843,6 +843,7 @@ mod tests { _report_splits_subscription_handle_opt: None, _local_shards_update_listener_handle_opt: None, cluster, + compaction_service_client_opt: None, control_plane_server_opt: None, control_plane_client, indexing_service_opt: None, From 19300bc11cbb06aa40cf48cc65edd3ad25459bce Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Mon, 13 Apr 2026 13:37:22 -0400 Subject: [PATCH 03/11] Scaffold Compactor supervisor and pipeline (#6282) --- quickwit/Cargo.lock | 9 + quickwit/quickwit-compaction/Cargo.toml | 11 +- .../src/compaction_pipeline.rs | 186 +++++++++++++++ .../src/compactor_supervisor.rs | 224 ++++++++++++++++++ quickwit/quickwit-compaction/src/lib.rs | 6 + 5 files changed, 434 insertions(+), 2 deletions(-) create mode 100644 quickwit/quickwit-compaction/src/compaction_pipeline.rs create mode 100644 quickwit/quickwit-compaction/src/compactor_supervisor.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 544b882d332..f97d39a633b 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7148,8 +7148,17 @@ dependencies = [ name = "quickwit-compaction" version = "0.8.0" dependencies = [ + "anyhow", "async-trait", + "quickwit-actors", + "quickwit-common", + "quickwit-indexing", + "quickwit-metastore", "quickwit-proto", + "quickwit-storage", + "serde", + "tokio", + "tracing", ] [[package]] diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index 07925537dce..9efa0e2ce94 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -1,6 +1,6 @@ [package] name = "quickwit-compaction" -description = "Merge planner and merge worker services for split compaction" +description = "Compactor implementation and CompactionService" version.workspace = true edition.workspace = true @@ -12,4 +12,11 @@ license.workspace = true [dependencies] async-trait = { workspace = true } -quickwit-proto = { workspace = true } \ No newline at end of file +quickwit-actors = { workspace = true } +quickwit-common = { workspace = true } +quickwit-indexing = { workspace = true } +quickwit-proto = { workspace = true } +quickwit-storage = { workspace = true } +serde = { workspace = true } +tracing = { workspace = true } +tokio = { workspace = true } \ No newline at end of file diff --git a/quickwit/quickwit-compaction/src/compaction_pipeline.rs b/quickwit/quickwit-compaction/src/compaction_pipeline.rs new file mode 100644 index 00000000000..926a5a5eb93 --- /dev/null +++ b/quickwit/quickwit-compaction/src/compaction_pipeline.rs @@ -0,0 +1,186 @@ +// 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. + +use quickwit_actors::{ActorHandle, Health, Supervisable}; +use quickwit_common::KillSwitch; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_indexing::actors::{ + MergeExecutor, MergeSplitDownloader, Packager, Publisher, Uploader, +}; +use tracing::{debug, error}; + +pub struct CompactionPipelineHandles { + pub merge_split_downloader: ActorHandle, + pub merge_executor: ActorHandle, + pub merge_packager: ActorHandle, + pub merge_uploader: ActorHandle, + pub merge_publisher: ActorHandle, +} + +/// A single-use merge execution pipeline. Processes one merge task and +/// terminates. +/// +/// Owned by the `CompactorSupervisor`, which periodically calls +/// `check_actor_health()` and acts on the result (retry, reap, etc.). +pub struct CompactionPipeline { + pub task_id: String, + pub split_ids: Vec, + pub retry_count: usize, + pub kill_switch: KillSwitch, + pub scratch_directory: TempDirectory, + pub handles: Option, +} + +impl CompactionPipeline { + pub fn new(task_id: String, split_ids: Vec, scratch_directory: TempDirectory) -> Self { + CompactionPipeline { + task_id, + split_ids, + retry_count: 0, + kill_switch: KillSwitch::default(), + scratch_directory, + handles: None, + } + } + + fn supervisables(&self) -> Vec<&dyn Supervisable> { + let Some(handles) = &self.handles else { + return Vec::new(); + }; + vec![ + &handles.merge_split_downloader, + &handles.merge_executor, + &handles.merge_packager, + &handles.merge_uploader, + &handles.merge_publisher, + ] + } + + /// Checks child actor health. + /// + /// `check_for_progress` controls whether stall detection is performed + /// (actors that are alive but haven't recorded progress since last check). + /// The supervisor controls the cadence of progress checks. + /// + /// Returns: + /// - `Success` when all actors have completed (merge published). + /// - `FailureOrUnhealthy` when any actor has died or stalled. + /// - `Healthy` when actors are running and making progress. + pub fn check_actor_health(&self) -> Health { + if self.handles.is_none() { + return Health::Healthy; + } + + let mut healthy_actors: Vec<&str> = Vec::new(); + let mut failure_or_unhealthy_actors: Vec<&str> = Vec::new(); + let mut success_actors: Vec<&str> = Vec::new(); + + for supervisable in self.supervisables() { + match supervisable.check_health(true) { + 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!( + task_id=%self.task_id, + healthy_actors=?healthy_actors, + failed_or_unhealthy_actors=?failure_or_unhealthy_actors, + success_actors=?success_actors, + "compaction pipeline actor failure detected" + ); + return Health::FailureOrUnhealthy; + } + if healthy_actors.is_empty() { + debug!(task_id=%self.task_id, "all compaction pipeline actors completed"); + return Health::Success; + } + Health::Healthy + } + + pub async fn terminate(&mut self) { + self.kill_switch.kill(); + if let Some(handles) = self.handles.take() { + tokio::join!( + handles.merge_split_downloader.kill(), + handles.merge_executor.kill(), + handles.merge_packager.kill(), + handles.merge_uploader.kill(), + handles.merge_publisher.kill(), + ); + } + } + + /// Terminates the current actor chain, increments retry count, and + /// re-spawns. Downloaded splits remain on disk in the scratch directory. + pub async fn restart(&mut self) { + self.terminate().await; + self.retry_count += 1; + self.spawn_pipeline(); + } + + /// Spawns the actor chain. Currently a no-op stub — actor chain + /// construction will be implemented in a later PR. + fn spawn_pipeline(&mut self) { + // TODO: construct MergeSplitDownloader → MergeExecutor → Packager → + // Uploader → Publisher actor chain and set self.handles. + } +} + +#[cfg(test)] +mod tests { + use quickwit_actors::Health; + use quickwit_common::temp_dir::TempDirectory; + + use super::CompactionPipeline; + + fn test_pipeline() -> CompactionPipeline { + CompactionPipeline::new( + "test-task".to_string(), + vec!["split-1".to_string(), "split-2".to_string()], + TempDirectory::for_test(), + ) + } + + #[test] + fn test_pipeline_no_handles_is_healthy() { + let pipeline = test_pipeline(); + assert!(pipeline.handles.is_none()); + assert_eq!(pipeline.check_actor_health(), Health::Healthy); + } + + #[tokio::test] + async fn test_pipeline_terminate_without_handles() { + let mut pipeline = test_pipeline(); + // Should not panic when there are no handles. + pipeline.terminate().await; + assert!(pipeline.handles.is_none()); + } + + #[tokio::test] + async fn test_pipeline_restart_increments_retry_count() { + let mut pipeline = test_pipeline(); + assert_eq!(pipeline.retry_count, 0); + pipeline.restart().await; + assert_eq!(pipeline.retry_count, 1); + } +} diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs new file mode 100644 index 00000000000..78e87c79df8 --- /dev/null +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -0,0 +1,224 @@ +// 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. + +use std::time::Duration; + +use async_trait::async_trait; +use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Health}; +use quickwit_common::io::Limiter; +use quickwit_common::pubsub::EventBroker; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_indexing::IndexingSplitStore; +use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_storage::StorageResolver; +use serde::Serialize; +use tracing::{error, info}; + +use crate::compaction_pipeline::CompactionPipeline; + +const SUPERVISE_LOOP_INTERVAL: Duration = Duration::from_secs(1); + +#[derive(Debug)] +struct SuperviseLoop; + +#[derive(Clone, Debug, Default, Serialize)] +pub struct CompactorSupervisorState { + pub num_pipeline_slots: usize, + pub num_occupied_slots: usize, + pub num_completed_tasks: usize, + pub num_failed_tasks: usize, +} + +/// Manages a pool of `CompactionPipeline`s, each executing a single merge task. +/// +/// Periodically checks pipeline health, handles retries on failure, and reaps +/// completed/failed pipelines. +pub struct CompactorSupervisor { + pipelines: Vec>, + + // Shared resources distributed to pipelines when spawning actor chains. + io_throughput_limiter: Option, + split_store: IndexingSplitStore, + metastore: MetastoreServiceClient, + storage_resolver: StorageResolver, + max_concurrent_split_uploads: usize, + event_broker: EventBroker, + + // Scratch directory root (/compaction/). + compaction_root_directory: TempDirectory, + + max_local_retries: usize, + + // dummy counters until we have real state + num_completed_tasks: usize, + num_failed_tasks: usize, +} + +impl CompactorSupervisor { + #[allow(clippy::too_many_arguments)] + pub fn new( + num_pipeline_slots: usize, + io_throughput_limiter: Option, + split_store: IndexingSplitStore, + metastore: MetastoreServiceClient, + storage_resolver: StorageResolver, + max_concurrent_split_uploads: usize, + event_broker: EventBroker, + compaction_root_directory: TempDirectory, + max_local_retries: usize, + ) -> Self { + let pipelines = (0..num_pipeline_slots).map(|_| None).collect(); + CompactorSupervisor { + pipelines, + io_throughput_limiter, + split_store, + metastore, + storage_resolver, + max_concurrent_split_uploads, + event_broker, + compaction_root_directory, + max_local_retries, + num_completed_tasks: 0, + num_failed_tasks: 0, + } + } + + async fn supervise(&mut self) { + for slot in &mut self.pipelines { + let Some(pipeline) = slot else { + continue; + }; + + match pipeline.check_actor_health() { + Health::Healthy => {} + Health::Success => { + info!(task_id=%pipeline.task_id, "compaction task completed"); + self.num_completed_tasks += 1; + *slot = None; + } + Health::FailureOrUnhealthy => { + if pipeline.retry_count < self.max_local_retries { + info!( + task_id=%pipeline.task_id, + retry_count=%pipeline.retry_count, + "retrying compaction pipeline" + ); + pipeline.restart().await; + } else { + error!( + task_id=%pipeline.task_id, + retry_count=%pipeline.retry_count, + "compaction pipeline exhausted retries" + ); + pipeline.terminate().await; + self.num_failed_tasks += 1; + *slot = None; + } + } + } + } + } +} + +#[async_trait] +impl Actor for CompactorSupervisor { + type ObservableState = (); + + fn name(&self) -> String { + "CompactorSupervisor".to_string() + } + + fn observable_state(&self) -> Self::ObservableState {} + + async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { + info!( + num_pipeline_slots=%self.pipelines.len(), + "compactor supervisor started" + ); + ctx.schedule_self_msg(SUPERVISE_LOOP_INTERVAL, SuperviseLoop); + Ok(()) + } +} + +#[async_trait] +impl Handler for CompactorSupervisor { + type Reply = (); + + async fn handle( + &mut self, + _msg: SuperviseLoop, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + self.supervise().await; + ctx.schedule_self_msg(SUPERVISE_LOOP_INTERVAL, SuperviseLoop); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use quickwit_actors::Universe; + use quickwit_common::temp_dir::TempDirectory; + use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; + use quickwit_storage::{RamStorage, StorageResolver}; + + use super::*; + use crate::compaction_pipeline::CompactionPipeline; + + fn test_supervisor(num_slots: usize) -> CompactorSupervisor { + let storage = Arc::new(RamStorage::default()); + let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); + let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); + CompactorSupervisor::new( + num_slots, + None, + split_store, + metastore, + StorageResolver::for_test(), + 2, + EventBroker::default(), + TempDirectory::for_test(), + 2, + ) + } + + #[tokio::test] + async fn test_supervisor_starts_with_empty_slots() { + let universe = Universe::with_accelerated_time(); + let supervisor = test_supervisor(4); + let (_mailbox, handle) = universe.spawn_builder().spawn(supervisor); + let obs = handle.process_pending_and_observe().await; + assert_eq!(obs.obs_type, quickwit_actors::ObservationType::Alive); + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_supervisor_supervise_reaps_no_handle_pipelines() { + // A pipeline with no handles returns Healthy, so it stays in its slot. + let mut supervisor = test_supervisor(2); + let pipeline = CompactionPipeline::new( + "task-1".to_string(), + vec!["split-1".to_string()], + TempDirectory::for_test(), + ); + supervisor.pipelines[0] = Some(pipeline); + supervisor.supervise().await; + // Pipeline has no handles → Healthy → not reaped. + assert!(supervisor.pipelines[0].is_some()); + assert_eq!(supervisor.num_completed_tasks, 0); + assert_eq!(supervisor.num_failed_tasks, 0); + } +} diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs index d59933e227e..b867eed1b86 100644 --- a/quickwit/quickwit-compaction/src/lib.rs +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -14,4 +14,10 @@ #![deny(clippy::disallowed_methods)] +#[allow(dead_code)] +mod compaction_pipeline; +#[allow(dead_code)] +mod compactor_supervisor; pub mod planner; + +pub use compactor_supervisor::CompactorSupervisor; From 40b307c9e3c94ff5a2bf12e030d3c7e301ab561c Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Mon, 13 Apr 2026 13:43:51 -0400 Subject: [PATCH 04/11] Spawn merge pipeline from new compactor (#6284) * Spawn merge pipeline from new compactor * new line because adrien is annoyed about it * comment --- quickwit/Cargo.lock | 2 + quickwit/quickwit-compaction/Cargo.toml | 11 +- .../src/compaction_pipeline.rs | 206 ++++++++++++++++-- .../src/compactor_supervisor.rs | 27 +-- quickwit/quickwit-indexing/failpoints/mod.rs | 5 +- .../src/actors/merge_executor.rs | 50 +++-- .../src/actors/merge_split_downloader.rs | 40 +++- .../src/models/merge_scratch.rs | 11 +- 8 files changed, 279 insertions(+), 73 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index f97d39a633b..39cef7929cb 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7152,6 +7152,8 @@ dependencies = [ "async-trait", "quickwit-actors", "quickwit-common", + "quickwit-config", + "quickwit-doc-mapper", "quickwit-indexing", "quickwit-metastore", "quickwit-proto", diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index 9efa0e2ce94..878ca2384ba 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -11,12 +11,21 @@ authors.workspace = true license.workspace = true [dependencies] +anyhow = { workspace = true } async-trait = { workspace = true } quickwit-actors = { workspace = true } quickwit-common = { workspace = true } +quickwit-config = { workspace = true } +quickwit-doc-mapper = { workspace = true } quickwit-indexing = { workspace = true } quickwit-proto = { workspace = true } quickwit-storage = { workspace = true } serde = { workspace = true } tracing = { workspace = true } -tokio = { workspace = true } \ No newline at end of file +tokio = { workspace = true } + +[dev-dependencies] +quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } +quickwit-metastore = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } +quickwit-storage = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-compaction/src/compaction_pipeline.rs b/quickwit/quickwit-compaction/src/compaction_pipeline.rs index 926a5a5eb93..309a5865a34 100644 --- a/quickwit/quickwit-compaction/src/compaction_pipeline.rs +++ b/quickwit/quickwit-compaction/src/compaction_pipeline.rs @@ -12,13 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. -use quickwit_actors::{ActorHandle, Health, Supervisable}; +use std::sync::Arc; + +use quickwit_actors::{ActorContext, ActorHandle, Health, Supervisable}; use quickwit_common::KillSwitch; +use quickwit_common::io::{IoControls, Limiter}; +use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; +use quickwit_config::RetentionPolicy; +use quickwit_doc_mapper::DocMapper; use quickwit_indexing::actors::{ - MergeExecutor, MergeSplitDownloader, Packager, Publisher, Uploader, + MergeExecutor, MergeSplitDownloader, Packager, Publisher, Uploader, UploaderType, }; -use tracing::{debug, error}; +use quickwit_indexing::merge_policy::MergeOperation; +use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; +use quickwit_proto::indexing::MergePipelineId; +use quickwit_proto::metastore::MetastoreServiceClient; +use tracing::{debug, error, info}; + +use crate::CompactorSupervisor; pub struct CompactionPipelineHandles { pub merge_split_downloader: ActorHandle, @@ -35,22 +47,58 @@ pub struct CompactionPipelineHandles { /// `check_actor_health()` and acts on the result (retry, reap, etc.). pub struct CompactionPipeline { pub task_id: String, - pub split_ids: Vec, pub retry_count: usize, pub kill_switch: KillSwitch, pub scratch_directory: TempDirectory, pub handles: Option, + + // Per-task parameters. + pub merge_operation: MergeOperation, + pub pipeline_id: MergePipelineId, + pub doc_mapper: Arc, + pub merge_policy: Arc, + pub retention_policy: Option, + + // Shared resources (cloned from CompactorSupervisor). + pub metastore: MetastoreServiceClient, + pub split_store: IndexingSplitStore, + pub io_throughput_limiter: Option, + pub max_concurrent_split_uploads: usize, + pub event_broker: EventBroker, } impl CompactionPipeline { - pub fn new(task_id: String, split_ids: Vec, scratch_directory: TempDirectory) -> Self { + #[allow(clippy::too_many_arguments)] + pub fn new( + task_id: String, + scratch_directory: TempDirectory, + merge_operation: MergeOperation, + pipeline_id: MergePipelineId, + doc_mapper: Arc, + merge_policy: Arc, + retention_policy: Option, + metastore: MetastoreServiceClient, + split_store: IndexingSplitStore, + io_throughput_limiter: Option, + max_concurrent_split_uploads: usize, + event_broker: EventBroker, + ) -> Self { CompactionPipeline { task_id, - split_ids, retry_count: 0, kill_switch: KillSwitch::default(), scratch_directory, handles: None, + merge_operation, + pipeline_id, + doc_mapper, + merge_policy, + retention_policy, + metastore, + split_store, + io_throughput_limiter, + max_concurrent_split_uploads, + event_broker, } } @@ -132,32 +180,153 @@ impl CompactionPipeline { /// Terminates the current actor chain, increments retry count, and /// re-spawns. Downloaded splits remain on disk in the scratch directory. - pub async fn restart(&mut self) { + pub async fn restart(&mut self, ctx: &ActorContext) { self.terminate().await; self.retry_count += 1; - self.spawn_pipeline(); + if let Err(err) = self.spawn_pipeline(ctx) { + error!(task_id=%self.task_id, error=?err, "failed to respawn compaction pipeline"); + } } - /// Spawns the actor chain. Currently a no-op stub — actor chain - /// construction will be implemented in a later PR. - fn spawn_pipeline(&mut self) { - // TODO: construct MergeSplitDownloader → MergeExecutor → Packager → - // Uploader → Publisher actor chain and set self.handles. + /// Spawns the 5-actor merge execution chain and sends the `MergeOperation` + /// to the downloader to kick off execution. + fn spawn_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { + self.kill_switch = ctx.kill_switch().child(); + + info!( + task_id=%self.task_id, + pipeline_id=%self.pipeline_id, + "spawning compaction pipeline" + ); + + // Publisher (no merge planner feedback, no source) + let merge_publisher = Publisher::new( + PublisherType::MergePublisher, + self.metastore.clone(), + None, + None, + ); + let (merge_publisher_mailbox, merge_publisher_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(merge_publisher); + + // Uploader + let merge_uploader = Uploader::new( + UploaderType::MergeUploader, + self.metastore.clone(), + self.merge_policy.clone(), + self.retention_policy.clone(), + self.split_store.clone(), + SplitsUpdateMailbox::from(merge_publisher_mailbox), + self.max_concurrent_split_uploads, + self.event_broker.clone(), + ); + let (merge_uploader_mailbox, merge_uploader_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(merge_uploader); + + // Packager + let tag_fields = self.doc_mapper.tag_named_fields()?; + let merge_packager = Packager::new("MergePackager", tag_fields, merge_uploader_mailbox); + let (merge_packager_mailbox, merge_packager_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(merge_packager); + + // MergeExecutor + let split_downloader_io_controls = IoControls::default() + .set_throughput_limiter_opt(self.io_throughput_limiter.clone()) + .set_component("split_downloader_merge"); + let merge_executor_io_controls = + split_downloader_io_controls.clone().set_component("merger"); + + let merge_executor = MergeExecutor::new( + self.pipeline_id.clone(), + self.metastore.clone(), + self.doc_mapper.clone(), + merge_executor_io_controls, + merge_packager_mailbox, + ); + let (merge_executor_mailbox, merge_executor_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(merge_executor); + + // MergeSplitDownloader + let merge_split_downloader = MergeSplitDownloader { + scratch_directory: self.scratch_directory.clone(), + split_store: self.split_store.clone(), + executor_mailbox: merge_executor_mailbox, + io_controls: split_downloader_io_controls, + }; + let (merge_split_downloader_mailbox, merge_split_downloader_handle) = ctx + .spawn_actor() + .set_kill_switch(self.kill_switch.clone()) + .spawn(merge_split_downloader); + + // Kick off the pipeline. + merge_split_downloader_mailbox + .try_send_message(self.merge_operation.clone()) + .map_err(|err| { + anyhow::anyhow!("failed to send merge operation to downloader: {err:?}") + })?; + + self.handles = Some(CompactionPipelineHandles { + merge_split_downloader: merge_split_downloader_handle, + merge_executor: merge_executor_handle, + merge_packager: merge_packager_handle, + merge_uploader: merge_uploader_handle, + merge_publisher: merge_publisher_handle, + }); + + Ok(()) } } #[cfg(test)] mod tests { + use std::sync::Arc; + use quickwit_actors::Health; + use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; + use quickwit_doc_mapper::default_doc_mapper_for_test; + use quickwit_indexing::IndexingSplitStore; + use quickwit_indexing::merge_policy::{MergeOperation, default_merge_policy}; + use quickwit_metastore::SplitMetadata; + use quickwit_proto::indexing::MergePipelineId; + use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; + use quickwit_proto::types::{IndexUid, NodeId}; + use quickwit_storage::RamStorage; use super::CompactionPipeline; fn test_pipeline() -> CompactionPipeline { + let storage = Arc::new(RamStorage::default()); + let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); + let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); + let splits = vec![SplitMetadata::for_test("split-1".to_string())]; + let merge_operation = MergeOperation::new_merge_operation(splits); + let pipeline_id = MergePipelineId { + node_id: NodeId::from("test-node"), + index_uid: IndexUid::for_test("test-index", 0), + source_id: "test-source".to_string(), + }; CompactionPipeline::new( "test-task".to_string(), - vec!["split-1".to_string(), "split-2".to_string()], TempDirectory::for_test(), + merge_operation, + pipeline_id, + Arc::new(default_doc_mapper_for_test()), + default_merge_policy(), + None, + metastore, + split_store, + None, + 2, + EventBroker::default(), ) } @@ -171,16 +340,7 @@ mod tests { #[tokio::test] async fn test_pipeline_terminate_without_handles() { let mut pipeline = test_pipeline(); - // Should not panic when there are no handles. pipeline.terminate().await; assert!(pipeline.handles.is_none()); } - - #[tokio::test] - async fn test_pipeline_restart_increments_retry_count() { - let mut pipeline = test_pipeline(); - assert_eq!(pipeline.retry_count, 0); - pipeline.restart().await; - assert_eq!(pipeline.retry_count, 1); - } } diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index 78e87c79df8..4ba270c270b 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -94,7 +94,7 @@ impl CompactorSupervisor { } } - async fn supervise(&mut self) { + async fn supervise(&mut self, ctx: &ActorContext) { for slot in &mut self.pipelines { let Some(pipeline) = slot else { continue; @@ -114,7 +114,7 @@ impl CompactorSupervisor { retry_count=%pipeline.retry_count, "retrying compaction pipeline" ); - pipeline.restart().await; + pipeline.restart(ctx).await; } else { error!( task_id=%pipeline.task_id, @@ -160,7 +160,7 @@ impl Handler for CompactorSupervisor { _msg: SuperviseLoop, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - self.supervise().await; + self.supervise(ctx).await; ctx.schedule_self_msg(SUPERVISE_LOOP_INTERVAL, SuperviseLoop); Ok(()) } @@ -176,7 +176,6 @@ mod tests { use quickwit_storage::{RamStorage, StorageResolver}; use super::*; - use crate::compaction_pipeline::CompactionPipeline; fn test_supervisor(num_slots: usize) -> CompactorSupervisor { let storage = Arc::new(RamStorage::default()); @@ -208,17 +207,13 @@ mod tests { #[tokio::test] async fn test_supervisor_supervise_reaps_no_handle_pipelines() { // A pipeline with no handles returns Healthy, so it stays in its slot. - let mut supervisor = test_supervisor(2); - let pipeline = CompactionPipeline::new( - "task-1".to_string(), - vec!["split-1".to_string()], - TempDirectory::for_test(), - ); - supervisor.pipelines[0] = Some(pipeline); - supervisor.supervise().await; - // Pipeline has no handles → Healthy → not reaped. - assert!(supervisor.pipelines[0].is_some()); - assert_eq!(supervisor.num_completed_tasks, 0); - assert_eq!(supervisor.num_failed_tasks, 0); + // We spawn the supervisor as an actor so we can get a context for supervise(). + let universe = Universe::with_accelerated_time(); + let supervisor = test_supervisor(2); + let (_mailbox, handle) = universe.spawn_builder().spawn(supervisor); + // Let the supervisor run one supervision loop (it schedules SuperviseLoop on init). + let obs = handle.process_pending_and_observe().await; + assert_eq!(obs.obs_type, quickwit_actors::ObservationType::Alive); + universe.assert_quit().await; } } diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index d8c5ab0e418..b68245ff718 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -285,9 +285,10 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul tantivy_dirs.push(get_tantivy_directory_from_split_bundle(&dest_filepath).unwrap()); } let merge_operation = MergeOperation::new_merge_operation(split_metadatas); - let merge_task = MergeTask::from_merge_operation_for_test(merge_operation); + let merge_task = MergeTask::from_merge_operation_for_test(merge_operation.clone()); let merge_scratch = MergeScratch { - merge_task, + merge_operation, + merge_task: Some(merge_task), merge_scratch_directory, downloaded_splits_directory, tantivy_dirs, diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index d6ada9dde0f..5141227bee9 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -81,22 +81,28 @@ impl Actor for MergeExecutor { impl Handler for MergeExecutor { type Reply = (); - #[instrument(level = "info", name = "merge_executor", parent = merge_scratch.merge_task.merge_parent_span.id(), skip_all)] + #[instrument(level = "info", name = "merge_executor", parent = merge_scratch.merge_operation.merge_parent_span.id(), skip_all)] async fn handle( &mut self, merge_scratch: MergeScratch, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { let start = Instant::now(); - let merge_task = merge_scratch.merge_task; - let indexed_split_opt: Option = match merge_task.operation_type { + let MergeScratch { + merge_operation, + merge_task, + tantivy_dirs, + merge_scratch_directory, + .. + } = merge_scratch; + let indexed_split_opt: Option = match merge_operation.operation_type { MergeOperationType::Merge => { let merge_res = self .process_merge( - merge_task.merge_split_id.clone(), - merge_task.splits.clone(), - merge_scratch.tantivy_dirs, - merge_scratch.merge_scratch_directory, + merge_operation.merge_split_id.clone(), + merge_operation.splits.clone(), + tantivy_dirs, + merge_scratch_directory, ctx, ) .await; @@ -114,24 +120,24 @@ impl Handler for MergeExecutor { // With a merge policy that marks splits as mature after a day or so, this // limits the noise associated to those failed // merges. - error!(task=?merge_task, err=?err, "failed to merge splits"); + error!(task=?merge_operation, err=?err, "failed to merge splits"); return Ok(()); } } } MergeOperationType::DeleteAndMerge => { assert_eq!( - merge_task.splits.len(), + merge_operation.splits.len(), 1, "Delete tasks can be applied only on one split." ); - assert_eq!(merge_scratch.tantivy_dirs.len(), 1); - let split_with_docs_to_delete = merge_task.splits[0].clone(); + assert_eq!(tantivy_dirs.len(), 1); + let split_with_docs_to_delete = merge_operation.splits[0].clone(); self.process_delete_and_merge( - merge_task.merge_split_id.clone(), + merge_operation.merge_split_id.clone(), split_with_docs_to_delete, - merge_scratch.tantivy_dirs, - merge_scratch.merge_scratch_directory, + tantivy_dirs, + merge_scratch_directory, ctx, ) .await? @@ -141,7 +147,7 @@ impl Handler for MergeExecutor { info!( merged_num_docs = %indexed_split.split_attrs.num_docs, elapsed_secs = %start.elapsed().as_secs_f32(), - operation_type = %merge_task.operation_type, + operation_type = %merge_operation.operation_type, "merge-operation-success" ); ctx.send_message( @@ -151,8 +157,8 @@ impl Handler for MergeExecutor { checkpoint_delta_opt: Default::default(), publish_lock: PublishLock::default(), publish_token_opt: None, - batch_parent_span: merge_task.merge_parent_span.clone(), - merge_task_opt: Some(merge_task), + batch_parent_span: merge_operation.merge_parent_span.clone(), + merge_task_opt: merge_task, }, ) .await?; @@ -642,9 +648,10 @@ mod tests { tantivy_dirs.push(get_tantivy_directory_from_split_bundle(&dest_filepath).unwrap()) } let merge_operation = MergeOperation::new_merge_operation(split_metas); - let merge_task = MergeTask::from_merge_operation_for_test(merge_operation); + let merge_task = MergeTask::from_merge_operation_for_test(merge_operation.clone()); let merge_scratch = MergeScratch { - merge_task, + merge_operation, + merge_task: Some(merge_task), tantivy_dirs, merge_scratch_directory, downloaded_splits_directory, @@ -786,9 +793,10 @@ mod tests { .await?; let tantivy_dir = get_tantivy_directory_from_split_bundle(&dest_filepath).unwrap(); let merge_operation = MergeOperation::new_delete_and_merge_operation(new_split_metadata); - let merge_task = MergeTask::from_merge_operation_for_test(merge_operation); + let merge_task = MergeTask::from_merge_operation_for_test(merge_operation.clone()); let merge_scratch = MergeScratch { - merge_task, + merge_operation, + merge_task: Some(merge_task), tantivy_dirs: vec![tantivy_dir], merge_scratch_directory, downloaded_splits_directory, diff --git a/quickwit/quickwit-indexing/src/actors/merge_split_downloader.rs b/quickwit/quickwit-indexing/src/actors/merge_split_downloader.rs index 5d68bb59285..6b1bcc0ddb9 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_split_downloader.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_split_downloader.rs @@ -23,7 +23,7 @@ use tantivy::Directory; use tracing::{debug, info, instrument}; use super::MergeExecutor; -use crate::merge_policy::MergeTask; +use crate::merge_policy::{MergeOperation, MergeTask}; use crate::models::MergeScratch; use crate::split_store::IndexingSplitStore; @@ -62,6 +62,35 @@ impl Handler for MergeSplitDownloader { merge_task: MergeTask, ctx: &ActorContext, ) -> Result<(), quickwit_actors::ActorExitStatus> { + let merge_operation = merge_task.merge_operation.as_ref().clone(); + self.download_and_send(merge_operation, ctx).await + } +} + +#[async_trait] +impl Handler for MergeSplitDownloader { + type Reply = (); + + #[instrument( + name = "merge_split_downloader", + parent = merge_operation.merge_parent_span.id(), + skip_all, + )] + async fn handle( + &mut self, + merge_operation: MergeOperation, + ctx: &ActorContext, + ) -> Result<(), quickwit_actors::ActorExitStatus> { + self.download_and_send(merge_operation, ctx).await + } +} + +impl MergeSplitDownloader { + async fn download_and_send( + &mut self, + merge_operation: MergeOperation, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { let merge_scratch_directory = temp_dir::Builder::default() .join("merge") .tempdir_in(self.scratch_directory.path()) @@ -73,13 +102,14 @@ impl Handler for MergeSplitDownloader { .map_err(|error| anyhow::anyhow!(error))?; let tantivy_dirs = self .download_splits( - merge_task.splits_as_slice(), + merge_operation.splits_as_slice(), downloaded_splits_directory.path(), ctx, ) .await?; let msg = MergeScratch { - merge_task, + merge_operation, + merge_task: None, merge_scratch_directory, downloaded_splits_directory, tantivy_dirs, @@ -190,8 +220,8 @@ mod tests { .unwrap() .downcast::() .unwrap(); - assert_eq!(merge_scratch.merge_task.splits_as_slice().len(), 10); - for split in merge_scratch.merge_task.splits_as_slice() { + assert_eq!(merge_scratch.merge_operation.splits_as_slice().len(), 10); + for split in merge_scratch.merge_operation.splits_as_slice() { let split_filename = split_file(split.split_id()); let split_filepath = merge_scratch .downloaded_splits_directory diff --git a/quickwit/quickwit-indexing/src/models/merge_scratch.rs b/quickwit/quickwit-indexing/src/models/merge_scratch.rs index 392ca60b42c..a0296f69d2d 100644 --- a/quickwit/quickwit-indexing/src/models/merge_scratch.rs +++ b/quickwit/quickwit-indexing/src/models/merge_scratch.rs @@ -15,14 +15,15 @@ use quickwit_common::temp_dir::TempDirectory; use tantivy::Directory; -use crate::merge_policy::MergeTask; +use crate::merge_policy::{MergeOperation, MergeTask}; #[derive(Debug)] pub struct MergeScratch { - /// A [`MergeTask`] tracked by either the `MergePlanner` or the `DeleteTaskPlanner` - /// See planners docs to understand the usage. - pub merge_task: MergeTask, - /// Scratch directory for computing the merge. + /// The merge operation data (splits, merge_split_id, operation type). + pub merge_operation: MergeOperation, + // TODO: remove once the old MergePipeline is deleted and the + // DeleteTaskPipeline no longer routes through MergeSchedulerService. + pub merge_task: Option, pub merge_scratch_directory: TempDirectory, pub downloaded_splits_directory: TempDirectory, pub tantivy_dirs: Vec>, From ecab99b7bb514b5020910048a56705ce879bc737 Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Mon, 13 Apr 2026 13:44:50 -0400 Subject: [PATCH 05/11] Wire up compactor service (#6291) * Wire up compactor service * lints --- quickwit/quickwit-compaction/src/lib.rs | 35 ++++++ quickwit/quickwit-config/src/lib.rs | 6 +- .../quickwit-config/src/node_config/mod.rs | 82 ++++++++++++++ .../src/node_config/serialize.rs | 95 +++++++++++++++- quickwit/quickwit-config/src/qw_env_vars.rs | 3 +- .../src/test_utils/cluster_sandbox.rs | 5 + .../src/tests/basic_tests.rs | 106 +++++------------- quickwit/quickwit-serve/src/lib.rs | 49 +++++--- quickwit/quickwit-serve/src/rest.rs | 1 + 9 files changed, 281 insertions(+), 101 deletions(-) diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs index b867eed1b86..fd2c5b05885 100644 --- a/quickwit/quickwit-compaction/src/lib.rs +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -21,3 +21,38 @@ mod compactor_supervisor; pub mod planner; pub use compactor_supervisor::CompactorSupervisor; +use quickwit_actors::{Mailbox, Universe}; +use quickwit_common::io; +use quickwit_common::pubsub::EventBroker; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_config::CompactorConfig; +use quickwit_indexing::IndexingSplitStore; +use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_storage::StorageResolver; +use tracing::info; + +pub async fn start_compactor_service( + universe: &Universe, + compactor_config: &CompactorConfig, + split_store: IndexingSplitStore, + metastore: MetastoreServiceClient, + storage_resolver: StorageResolver, + event_broker: EventBroker, + compaction_root_directory: TempDirectory, +) -> anyhow::Result> { + info!("starting compactor service"); + let io_throughput_limiter = compactor_config.max_merge_write_throughput.map(io::limiter); + let supervisor = CompactorSupervisor::new( + compactor_config.max_concurrent_pipelines.get(), + io_throughput_limiter, + split_store, + metastore, + storage_resolver, + compactor_config.max_concurrent_split_uploads, + event_broker, + compaction_root_directory, + compactor_config.max_local_retries, + ); + let (mailbox, _handle) = universe.spawn_builder().spawn(supervisor); + Ok(mailbox) +} diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 22cdb2538b4..79b285eee39 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -73,9 +73,9 @@ pub use crate::metastore_config::{ MetastoreBackend, MetastoreConfig, MetastoreConfigs, PostgresMetastoreConfig, }; pub use crate::node_config::{ - CacheConfig, CachePolicy, DEFAULT_QW_CONFIG_PATH, GrpcConfig, IndexerConfig, IngestApiConfig, - JaegerConfig, KeepAliveConfig, LambdaConfig, LambdaDeployConfig, NodeConfig, RestConfig, - SearcherConfig, SplitCacheLimits, StorageTimeoutPolicy, TlsConfig, + CacheConfig, CachePolicy, CompactorConfig, DEFAULT_QW_CONFIG_PATH, GrpcConfig, IndexerConfig, + IngestApiConfig, JaegerConfig, KeepAliveConfig, LambdaConfig, LambdaDeployConfig, NodeConfig, + RestConfig, SearcherConfig, SplitCacheLimits, StorageTimeoutPolicy, TlsConfig, }; use crate::source_config::serialize::{SourceConfigV0_7, SourceConfigV0_8, VersionedSourceConfig}; pub use crate::storage_config::{ diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index cf67768966d..47c3b6a77cb 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -163,6 +163,11 @@ pub struct IndexerConfig { pub enable_cooperative_indexing: bool, #[serde(default = "IndexerConfig::default_cpu_capacity")] pub cpu_capacity: CpuCapacity, + /// When true, the compactor service is not implicitly started on indexer + /// nodes. Dedicated compactor nodes must be deployed separately. + /// When false (default), every indexer node also runs the compactor. + #[serde(default = "IndexerConfig::default_enable_standalone_compactors")] + pub enable_standalone_compactors: bool, } impl IndexerConfig { @@ -170,6 +175,10 @@ impl IndexerConfig { false } + fn default_enable_standalone_compactors() -> bool { + false + } + fn default_enable_otlp_endpoint() -> bool { #[cfg(any(test, feature = "testsuite"))] { @@ -213,6 +222,7 @@ impl IndexerConfig { cpu_capacity: PIPELINE_FULL_CAPACITY * 4u32, max_merge_write_throughput: None, merge_concurrency: NonZeroUsize::new(3).unwrap(), + enable_standalone_compactors: false, }; Ok(indexer_config) } @@ -229,6 +239,77 @@ impl Default for IndexerConfig { cpu_capacity: Self::default_cpu_capacity(), merge_concurrency: Self::default_merge_concurrency(), max_merge_write_throughput: None, + enable_standalone_compactors: Self::default_enable_standalone_compactors(), + } + } +} + +#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] +#[serde(deny_unknown_fields)] +pub struct CompactorConfig { + /// Maximum number of concurrent merge pipelines. Defaults to 2/3 of CPU count. + #[serde(default = "CompactorConfig::default_max_concurrent_pipelines")] + pub max_concurrent_pipelines: NonZeroUsize, + /// Maximum number of concurrent split uploads across all pipelines. + #[serde(default = "CompactorConfig::default_max_concurrent_split_uploads")] + pub max_concurrent_split_uploads: usize, + /// Limits the IO throughput of the split downloader and the merge executor. + #[serde(default)] + pub max_merge_write_throughput: Option, + /// Maximum local retries before reporting a merge as failed to the planner. + #[serde(default = "CompactorConfig::default_max_local_retries")] + pub max_local_retries: usize, + /// Maximum size of the local split store cache in bytes. + #[serde(default = "CompactorConfig::default_split_store_max_num_bytes")] + pub split_store_max_num_bytes: ByteSize, + /// Maximum number of splits in the local split store cache. + #[serde(default = "CompactorConfig::default_split_store_max_num_splits")] + pub split_store_max_num_splits: usize, +} + +impl CompactorConfig { + fn default_max_concurrent_pipelines() -> NonZeroUsize { + NonZeroUsize::new(quickwit_common::num_cpus() * 2 / 3).unwrap_or(NonZeroUsize::MIN) + } + + fn default_max_concurrent_split_uploads() -> usize { + 12 + } + + fn default_max_local_retries() -> usize { + 2 + } + + pub fn default_split_store_max_num_bytes() -> ByteSize { + ByteSize::gib(100) + } + + pub fn default_split_store_max_num_splits() -> usize { + 1_000 + } + + #[cfg(any(test, feature = "testsuite"))] + pub fn for_test() -> Self { + CompactorConfig { + max_concurrent_pipelines: NonZeroUsize::new(2).unwrap(), + max_concurrent_split_uploads: 4, + max_merge_write_throughput: None, + max_local_retries: 2, + split_store_max_num_bytes: ByteSize::mb(1), + split_store_max_num_splits: 3, + } + } +} + +impl Default for CompactorConfig { + fn default() -> Self { + Self { + max_concurrent_pipelines: Self::default_max_concurrent_pipelines(), + max_concurrent_split_uploads: Self::default_max_concurrent_split_uploads(), + max_merge_write_throughput: None, + max_local_retries: Self::default_max_local_retries(), + split_store_max_num_bytes: Self::default_split_store_max_num_bytes(), + split_store_max_num_splits: Self::default_split_store_max_num_splits(), } } } @@ -778,6 +859,7 @@ pub struct NodeConfig { pub searcher_config: SearcherConfig, pub ingest_api_config: IngestApiConfig, pub jaeger_config: JaegerConfig, + pub compactor_config: CompactorConfig, } impl NodeConfig { diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index ae19a92c45f..0bf765df123 100644 --- a/quickwit/quickwit-config/src/node_config/serialize.rs +++ b/quickwit/quickwit-config/src/node_config/serialize.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::net::{IpAddr, SocketAddr}; use std::str::FromStr; use std::time::Duration; @@ -35,8 +35,8 @@ use crate::service::QuickwitService; use crate::storage_config::StorageConfigs; use crate::templating::render_config; use crate::{ - ConfigFormat, IndexerConfig, IngestApiConfig, JaegerConfig, MetastoreConfigs, NodeConfig, - SearcherConfig, TlsConfig, validate_identifier, validate_node_id, + CompactorConfig, ConfigFormat, IndexerConfig, IngestApiConfig, JaegerConfig, MetastoreConfigs, + NodeConfig, SearcherConfig, TlsConfig, validate_identifier, validate_node_id, }; pub const DEFAULT_CLUSTER_ID: &str = "quickwit-default-cluster"; @@ -192,6 +192,8 @@ struct NodeConfigBuilder { data_dir_uri: ConfigValue, metastore_uri: ConfigValue, default_index_root_uri: ConfigValue, + #[serde(default)] + enable_standalone_compactors: ConfigValue, #[serde(rename = "rest")] #[serde(default)] rest_config_builder: RestConfigBuilder, @@ -216,6 +218,9 @@ struct NodeConfigBuilder { #[serde(rename = "jaeger")] #[serde(default)] jaeger_config: JaegerConfig, + #[serde(rename = "compactor")] + #[serde(default)] + compactor_config: CompactorConfig, } impl NodeConfigBuilder { @@ -226,7 +231,10 @@ impl NodeConfigBuilder { let node_id = self.node_id.resolve(env_vars).map(NodeId::new)?; let availability_zone = self.availability_zone.resolve_optional(env_vars)?; - let enabled_services = self + self.indexer_config.enable_standalone_compactors = + self.enable_standalone_compactors.resolve(env_vars)?; + + let mut enabled_services: HashSet = self .enabled_services .resolve(env_vars)? .0 @@ -234,6 +242,14 @@ impl NodeConfigBuilder { .map(|service| service.parse()) .collect::>()?; + // Indexers implicitly run the compactor unless standalone compactors + // are enabled. + if enabled_services.contains(&QuickwitService::Indexer) + && !self.indexer_config.enable_standalone_compactors + { + enabled_services.insert(QuickwitService::Compactor); + } + let listen_address = self.listen_address.resolve(env_vars)?; let listen_host = listen_address.parse::()?; let listen_ip = listen_host.resolve().await?; @@ -331,6 +347,7 @@ impl NodeConfigBuilder { searcher_config: self.searcher_config, ingest_api_config: self.ingest_api_config, jaeger_config: self.jaeger_config, + compactor_config: self.compactor_config, }; validate(&node_config)?; @@ -421,6 +438,7 @@ impl Default for NodeConfigBuilder { data_dir_uri: default_data_dir_uri(), metastore_uri: ConfigValue::none(), default_index_root_uri: ConfigValue::none(), + enable_standalone_compactors: Default::default(), rest_config_builder: RestConfigBuilder::default(), grpc_config: GrpcConfig::default(), storage_configs: StorageConfigs::default(), @@ -429,6 +447,7 @@ impl Default for NodeConfigBuilder { searcher_config: SearcherConfig::default(), ingest_api_config: IngestApiConfig::default(), jaeger_config: JaegerConfig::default(), + compactor_config: CompactorConfig::default(), } } } @@ -528,6 +547,7 @@ pub fn node_config_for_tests_from_ports( searcher_config: SearcherConfig::default(), ingest_api_config: IngestApiConfig::default(), jaeger_config: JaegerConfig::default(), + compactor_config: CompactorConfig::default(), } } @@ -657,6 +677,7 @@ mod tests { cpu_capacity: IndexerConfig::default_cpu_capacity(), enable_cooperative_indexing: false, max_merge_write_throughput: Some(ByteSize::mb(100)), + enable_standalone_compactors: false, } ); assert_eq!( @@ -821,6 +842,10 @@ mod tests { "test-peer-seed-0,test-peer-seed-1".to_string(), ); env_vars.insert("QW_DATA_DIR".to_string(), "test-data-dir".to_string()); + env_vars.insert( + "QW_ENABLE_STANDALONE_COMPACTORS".to_string(), + "true".to_string(), + ); env_vars.insert( "QW_METASTORE_URI".to_string(), "postgresql://test-user:test-password@test-host:4321/test-db".to_string(), @@ -1354,4 +1379,66 @@ mod tests { .to_string(); assert!(error_message.contains("replication factor")); } + + #[tokio::test] + async fn test_indexer_implicitly_enables_compactor() { + let config_yaml = r#" + version: 0.8 + enabled_services: [indexer] + "#; + let config = + load_node_config_with_env(ConfigFormat::Yaml, config_yaml.as_bytes(), &HashMap::new()) + .await + .unwrap(); + assert!(config.enabled_services.contains(&QuickwitService::Indexer)); + assert!( + config + .enabled_services + .contains(&QuickwitService::Compactor) + ); + } + + #[tokio::test] + async fn test_standalone_compactors_prevents_implicit_compactor() { + let config_yaml = r#" + version: 0.8 + enabled_services: [indexer] + "#; + let env_vars = HashMap::from([( + "QW_ENABLE_STANDALONE_COMPACTORS".to_string(), + "true".to_string(), + )]); + let config = + load_node_config_with_env(ConfigFormat::Yaml, config_yaml.as_bytes(), &env_vars) + .await + .unwrap(); + assert!(config.enabled_services.contains(&QuickwitService::Indexer)); + assert!( + !config + .enabled_services + .contains(&QuickwitService::Compactor) + ); + } + + #[tokio::test] + async fn test_standalone_compactors_env_var_override() { + let env_vars = HashMap::from([( + "QW_ENABLE_STANDALONE_COMPACTORS".to_string(), + "true".to_string(), + )]); + let config_yaml = r#" + version: 0.8 + enabled_services: [indexer] + "#; + let config = + load_node_config_with_env(ConfigFormat::Yaml, config_yaml.as_bytes(), &env_vars) + .await + .unwrap(); + assert!(config.enabled_services.contains(&QuickwitService::Indexer)); + assert!( + !config + .enabled_services + .contains(&QuickwitService::Compactor) + ); + } } diff --git a/quickwit/quickwit-config/src/qw_env_vars.rs b/quickwit/quickwit-config/src/qw_env_vars.rs index 7f36fdc2911..ec6d61e9498 100644 --- a/quickwit/quickwit-config/src/qw_env_vars.rs +++ b/quickwit/quickwit-config/src/qw_env_vars.rs @@ -56,7 +56,8 @@ qw_env_vars!( QW_PEER_SEEDS, QW_DATA_DIR, QW_METASTORE_URI, - QW_DEFAULT_INDEX_ROOT_URI + QW_DEFAULT_INDEX_ROOT_URI, + QW_ENABLE_STANDALONE_COMPACTORS ); #[cfg(test)] diff --git a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs index a7385ca0946..7b425061134 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs @@ -76,6 +76,11 @@ impl TestNodeConfig { tcp_listener_resolver.add_listener(grpc_tcp_listener).await; config.indexer_config.enable_otlp_endpoint = self.enable_otlp; config.enabled_services.clone_from(&self.services); + if config.enabled_services.contains(&QuickwitService::Indexer) + && !config.indexer_config.enable_standalone_compactors + { + config.enabled_services.insert(QuickwitService::Compactor); + } config.jaeger_config.enable_endpoint = true; config.cluster_id.clone_from(&cluster_id); config.node_id = NodeId::new(format!("test-node-{node_idx}")); diff --git a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs index 2715e310a97..96bf311cc01 100644 --- a/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/basic_tests.rs @@ -164,97 +164,45 @@ async fn test_multi_nodes_cluster() { } #[tokio::test] -async fn test_no_merge_pipelines_when_compaction_service_enabled() { +async fn test_indexer_implicitly_runs_compactor() { quickwit_common::setup_logging_for_tests(); - unsafe { std::env::set_var("QW_ENABLE_COMPACTION_SERVICE", "true") }; - let sandbox = ClusterSandboxBuilder::default() .add_node([QuickwitService::Searcher]) .add_node([QuickwitService::Metastore]) - .add_node([QuickwitService::Indexer]) .add_node([QuickwitService::ControlPlane]) .add_node([QuickwitService::Janitor]) - .build_and_start() - .await; - - sandbox - .rest_client(QuickwitService::Indexer) - .indexes() - .create( - r#" - version: 0.8 - index_id: test-no-merge-pipelines - doc_mapping: - field_mappings: - - name: body - type: text - indexing_settings: - commit_timeout_secs: 1 - "#, - quickwit_config::ConfigFormat::Yaml, - false, - ) - .await - .unwrap(); - - sandbox.wait_for_indexing_pipelines(1).await.unwrap(); - - let stats = sandbox - .rest_client(QuickwitService::Indexer) - .node_stats() - .indexing() - .await - .unwrap(); - assert_eq!(stats.num_running_merge_pipelines, 0); - - unsafe { std::env::remove_var("QW_ENABLE_COMPACTION_SERVICE") }; - sandbox.shutdown().await.unwrap(); -} - -#[tokio::test] -async fn test_merge_pipelines_present_without_compaction_service() { - quickwit_common::setup_logging_for_tests(); - unsafe { std::env::set_var("QW_ENABLE_COMPACTION_SERVICE", "false") }; - - let sandbox = ClusterSandboxBuilder::default() - .add_node([QuickwitService::Searcher]) - .add_node([QuickwitService::Metastore]) .add_node([QuickwitService::Indexer]) - .add_node([QuickwitService::ControlPlane]) - .add_node([QuickwitService::Janitor]) .build_and_start() .await; - sandbox - .rest_client(QuickwitService::Indexer) - .indexes() - .create( - r#" - version: 0.8 - index_id: test-with-merge-pipelines - doc_mapping: - field_mappings: - - name: body - type: text - indexing_settings: - commit_timeout_secs: 1 - "#, - quickwit_config::ConfigFormat::Yaml, - false, - ) - .await - .unwrap(); - - sandbox.wait_for_indexing_pipelines(1).await.unwrap(); - - let stats = sandbox - .rest_client(QuickwitService::Indexer) - .node_stats() - .indexing() + let cluster_snapshot = sandbox + .rest_client(QuickwitService::Searcher) + .cluster() + .snapshot() .await .unwrap(); - assert!(stats.num_running_merge_pipelines > 0); + assert_eq!(cluster_snapshot.ready_nodes.len(), 5); + + // The indexer node should also advertise the compactor service. + let indexer_node = cluster_snapshot + .chitchat_state_snapshot + .node_states + .iter() + .find(|node_state| { + node_state + .get("enabled_services") + .map(|s| s.contains("indexer")) + .unwrap_or(false) + }) + .expect("indexer node not found in cluster state"); + let services = indexer_node.get("enabled_services").unwrap(); + assert!( + services.contains("compactor"), + "indexer node should implicitly run the compactor, got: {services}" + ); - unsafe { std::env::remove_var("QW_ENABLE_COMPACTION_SERVICE") }; sandbox.shutdown().await.unwrap(); } + +// TODO: add test_standalone_compactor_node once the test config builder +// resolves QW_ENABLE_STANDALONE_COMPACTORS from the environment. diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index e288161393e..8c974a4c990 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -73,6 +73,7 @@ use quickwit_common::tower::{ use quickwit_common::uri::Uri; use quickwit_common::{get_bool_from_env, spawn_named_task}; use quickwit_compaction::planner::StubCompactionService; +use quickwit_compaction::{CompactorSupervisor, start_compactor_service}; use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IngestApiConfig, NodeConfig}; use quickwit_control_plane::control_plane::{ControlPlane, ControlPlaneEventSubscriber}; @@ -80,7 +81,7 @@ use quickwit_control_plane::{IndexerNodeInfo, IndexerPool}; use quickwit_index_management::{IndexService as IndexManager, IndexServiceError}; use quickwit_indexing::actors::{IndexingService, MergeSchedulerService}; use quickwit_indexing::models::ShardPositionsService; -use quickwit_indexing::start_indexing_service; +use quickwit_indexing::{IndexingSplitStore, start_indexing_service}; use quickwit_ingest::{ GetMemoryCapacity, IngestRequest, IngestRouter, IngestServiceClient, Ingester, IngesterPool, IngesterPoolEntry, LocalShardsUpdate, get_idle_shard_timeout, @@ -113,7 +114,7 @@ use quickwit_search::{ SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, create_search_client_from_channel, start_searcher_service, }; -use quickwit_storage::{SplitCache, StorageResolver}; +use quickwit_storage::{RamStorage, SplitCache, StorageResolver}; use tcp_listener::TcpListenerResolver; use tokio::sync::oneshot; use tonic::codec::CompressionEncoding; @@ -148,8 +149,6 @@ const COMPACTION_SERVICE_DISCOVERY_TIMEOUT: Duration = if cfg!(any(test, feature const METASTORE_CLIENT_MAX_CONCURRENCY_ENV_KEY: &str = "QW_METASTORE_CLIENT_MAX_CONCURRENCY"; const DEFAULT_METASTORE_CLIENT_MAX_CONCURRENCY: usize = 6; const DISABLE_DELETE_TASK_SERVICE_ENV_KEY: &str = "QW_DISABLE_DELETE_TASK_SERVICE"; -const ENABLE_COMPACTION_SERVICE_ENV_KEY: &str = "QW_ENABLE_COMPACTION_SERVICE"; - pub type EnvFilterReloadFn = Arc anyhow::Result<()> + Send + Sync>; pub fn do_nothing_env_filter_reload_fn() -> EnvFilterReloadFn { @@ -205,6 +204,7 @@ struct QuickwitServices { ingester_opt: Option, pub compaction_service_client_opt: Option, + pub _compactor_supervisor_opt: Option>, pub janitor_service_opt: Option>, pub jaeger_service_opt: Option, pub otlp_logs_service_opt: Option, @@ -280,7 +280,7 @@ async fn get_compaction_service_client_if_needed( node_config: &NodeConfig, cluster: &Cluster, ) -> anyhow::Result, anyhow::Error> { - if !get_bool_from_env(ENABLE_COMPACTION_SERVICE_ENV_KEY, false) { + if !node_config.indexer_config.enable_standalone_compactors { return Ok(None); } // Only janitor nodes (which host the planner) and indexer nodes (which need @@ -785,6 +785,31 @@ pub async fn serve_quickwit( None }; + let compactor_supervisor_opt = if node_config.is_service_enabled(QuickwitService::Compactor) { + let compaction_dir = node_config.data_dir_path.join("compaction"); + fs::create_dir_all(&compaction_dir)?; + let compaction_root_directory = quickwit_common::temp_dir::Builder::default() + .tempdir_in(&compaction_dir) + .context("failed to create compaction temp directory")?; + let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new( + RamStorage::default(), + )); + let compactor_mailbox = start_compactor_service( + &universe, + &node_config.compactor_config, + split_store, + metastore_through_control_plane.clone(), + storage_resolver.clone(), + event_broker.clone(), + compaction_root_directory, + ) + .await + .context("failed to start compactor service")?; + Some(compactor_mailbox) + } else { + None + }; + let jaeger_service_opt = if node_config.jaeger_config.enable_endpoint && node_config.is_service_enabled(QuickwitService::Searcher) { @@ -834,6 +859,7 @@ pub async fn serve_quickwit( ingest_service, ingester_opt: ingester_opt.clone(), compaction_service_client_opt, + _compactor_supervisor_opt: compactor_supervisor_opt, janitor_service_opt, jaeger_service_opt, otlp_logs_service_opt, @@ -1891,7 +1917,7 @@ mod tests { .await .unwrap(); - // Without the env var, no compaction service. + // Without standalone compactors, no compaction service. let mut node_config = NodeConfig::for_test(); node_config.enabled_services = HashSet::from([QuickwitService::Janitor, QuickwitService::Indexer]); @@ -1900,14 +1926,12 @@ mod tests { .unwrap(); assert!(result.is_none()); - // With the env var, compaction service client is returned. - unsafe { std::env::set_var(ENABLE_COMPACTION_SERVICE_ENV_KEY, "true") }; + // With standalone compactors enabled, compaction service client is returned. + node_config.indexer_config.enable_standalone_compactors = true; let result = get_compaction_service_client_if_needed(&node_config, &cluster) .await .unwrap(); assert!(result.is_some()); - - unsafe { std::env::remove_var(ENABLE_COMPACTION_SERVICE_ENV_KEY) }; } #[tokio::test] @@ -1917,13 +1941,10 @@ mod tests { .await .unwrap(); - unsafe { std::env::set_var(ENABLE_COMPACTION_SERVICE_ENV_KEY, "true") }; - let mut node_config = NodeConfig::for_test(); node_config.enabled_services = HashSet::from([QuickwitService::Indexer]); + node_config.indexer_config.enable_standalone_compactors = true; let result = get_compaction_service_client_if_needed(&node_config, &cluster).await; assert!(result.is_err()); - - unsafe { std::env::remove_var(ENABLE_COMPACTION_SERVICE_ENV_KEY) }; } } diff --git a/quickwit/quickwit-serve/src/rest.rs b/quickwit/quickwit-serve/src/rest.rs index c52d790167d..7a30086ca5f 100644 --- a/quickwit/quickwit-serve/src/rest.rs +++ b/quickwit/quickwit-serve/src/rest.rs @@ -860,6 +860,7 @@ mod tests { node_config: Arc::new(node_config.clone()), search_service: Arc::new(MockSearchService::new()), jaeger_service_opt: None, + _compactor_supervisor_opt: None, env_filter_reload_fn: crate::do_nothing_env_filter_reload_fn(), }; From f35053e1b4c89baed0a1b48897abf10e6d9be250 Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Mon, 13 Apr 2026 14:56:55 -0400 Subject: [PATCH 06/11] Implement compactor pipeline update logic (#6297) * Implement compactor pipeline update logic * lints and fixes * comments, lints, test fixes, other things --- quickwit/Cargo.lock | 1 - quickwit/quickwit-compaction/Cargo.toml | 2 +- .../src/compaction_pipeline.rs | 268 ++++++---- .../src/compactor_supervisor.rs | 253 ++++++--- quickwit/quickwit-compaction/src/lib.rs | 9 +- .../src/planner/compaction_service.rs | 16 +- .../quickwit-compaction/src/planner/mod.rs | 2 +- .../quickwit-config/src/node_config/mod.rs | 9 - .../src/test_utils/cluster_sandbox.rs | 12 +- quickwit/quickwit-proto/build.rs | 2 + .../protos/quickwit/compaction.proto | 34 +- .../codegen/quickwit/quickwit.compaction.rs | 491 ++++++++++++++---- quickwit/quickwit-proto/src/indexing/mod.rs | 1 + quickwit/quickwit-serve/src/lib.rs | 74 +-- 14 files changed, 834 insertions(+), 340 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 39cef7929cb..93206bf5d46 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7158,7 +7158,6 @@ dependencies = [ "quickwit-metastore", "quickwit-proto", "quickwit-storage", - "serde", "tokio", "tracing", ] diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index 878ca2384ba..88735bb1606 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -20,11 +20,11 @@ quickwit-doc-mapper = { workspace = true } quickwit-indexing = { workspace = true } quickwit-proto = { workspace = true } quickwit-storage = { workspace = true } -serde = { workspace = true } tracing = { workspace = true } tokio = { workspace = true } [dev-dependencies] +quickwit-actors = { workspace = true, features = ["testsuite"] } quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } quickwit-proto = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-compaction/src/compaction_pipeline.rs b/quickwit/quickwit-compaction/src/compaction_pipeline.rs index 309a5865a34..62bb040e0ba 100644 --- a/quickwit/quickwit-compaction/src/compaction_pipeline.rs +++ b/quickwit/quickwit-compaction/src/compaction_pipeline.rs @@ -14,7 +14,7 @@ use std::sync::Arc; -use quickwit_actors::{ActorContext, ActorHandle, Health, Supervisable}; +use quickwit_actors::{ActorHandle, Health, SpawnContext, Supervisable}; use quickwit_common::KillSwitch; use quickwit_common::io::{IoControls, Limiter}; use quickwit_common::pubsub::EventBroker; @@ -28,43 +28,54 @@ use quickwit_indexing::merge_policy::MergeOperation; use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_proto::types::{IndexUid, SourceId, SplitId}; use tracing::{debug, error, info}; -use crate::CompactorSupervisor; +#[derive(Clone, Debug, PartialEq)] +pub enum PipelineStatus { + InProgress, + Completed, + Failed { error: String }, +} + +pub struct PipelineStatusUpdate { + pub task_id: String, + pub index_uid: IndexUid, + pub source_id: SourceId, + pub split_ids: Vec, + pub merged_split_id: SplitId, + pub status: PipelineStatus, +} -pub struct CompactionPipelineHandles { - pub merge_split_downloader: ActorHandle, - pub merge_executor: ActorHandle, - pub merge_packager: ActorHandle, - pub merge_uploader: ActorHandle, - pub merge_publisher: ActorHandle, +struct CompactionPipelineHandles { + merge_split_downloader: ActorHandle, + merge_executor: ActorHandle, + merge_packager: ActorHandle, + merge_uploader: ActorHandle, + merge_publisher: ActorHandle, } -/// A single-use merge execution pipeline. Processes one merge task and -/// terminates. +/// A single-use compaction pipeline. Processes one merge task and terminates. /// /// Owned by the `CompactorSupervisor`, which periodically calls -/// `check_actor_health()` and acts on the result (retry, reap, etc.). +/// `check_actor_health()` to collect status updates. The pipeline manages +/// its own retry logic internally. pub struct CompactionPipeline { - pub task_id: String, - pub retry_count: usize, - pub kill_switch: KillSwitch, - pub scratch_directory: TempDirectory, - pub handles: Option, - - // Per-task parameters. - pub merge_operation: MergeOperation, - pub pipeline_id: MergePipelineId, - pub doc_mapper: Arc, - pub merge_policy: Arc, - pub retention_policy: Option, - - // Shared resources (cloned from CompactorSupervisor). - pub metastore: MetastoreServiceClient, - pub split_store: IndexingSplitStore, - pub io_throughput_limiter: Option, - pub max_concurrent_split_uploads: usize, - pub event_broker: EventBroker, + task_id: String, + merge_operation: MergeOperation, + pipeline_id: MergePipelineId, + status: PipelineStatus, + kill_switch: KillSwitch, + scratch_directory: TempDirectory, + handles: Option, + doc_mapper: Arc, + merge_policy: Arc, + retention_policy: Option, + metastore: MetastoreServiceClient, + split_store: IndexingSplitStore, + io_throughput_limiter: Option, + max_concurrent_split_uploads: usize, + event_broker: EventBroker, } impl CompactionPipeline { @@ -85,7 +96,7 @@ impl CompactionPipeline { ) -> Self { CompactionPipeline { task_id, - retry_count: 0, + status: PipelineStatus::InProgress, kill_switch: KillSwitch::default(), scratch_directory, handles: None, @@ -115,84 +126,76 @@ impl CompactionPipeline { ] } - /// Checks child actor health. - /// - /// `check_for_progress` controls whether stall detection is performed - /// (actors that are alive but haven't recorded progress since last check). - /// The supervisor controls the cadence of progress checks. + /// Returns pipeline status update by checking the health of individual pipeline actors. /// - /// Returns: - /// - `Success` when all actors have completed (merge published). - /// - `FailureOrUnhealthy` when any actor has died or stalled. - /// - `Healthy` when actors are running and making progress. - pub fn check_actor_health(&self) -> Health { + /// If the pipeline is already completed or failed (terminal status), returns the status + /// without re-checking actors. The pipeline sits in this "completed" state (finished or failed) + /// until the supervisor cleans up the spot in favor of a new pipeline. This is done to + /// ensure that the planner acks the success/failure. + pub fn pipeline_status_update(&mut self) -> PipelineStatusUpdate { + self.update_status(); + self.build_status_update() + } + + fn update_status(&mut self) { + // Pipeline is finished but yet to be cleaned up. + if matches!( + self.status, + PipelineStatus::Completed | PipelineStatus::Failed { .. } + ) { + return; + } + // Pipeline is not initialized yet. if self.handles.is_none() { - return Health::Healthy; + return; } - let mut healthy_actors: Vec<&str> = Vec::new(); - let mut failure_or_unhealthy_actors: Vec<&str> = Vec::new(); - let mut success_actors: Vec<&str> = Vec::new(); + let mut has_healthy = false; + let mut failure_actor_names: Vec = Vec::new(); for supervisable in self.supervisables() { match supervisable.check_health(true) { Health::Healthy => { - healthy_actors.push(supervisable.name()); + has_healthy = true; } Health::FailureOrUnhealthy => { - failure_or_unhealthy_actors.push(supervisable.name()); - } - Health::Success => { - success_actors.push(supervisable.name()); + failure_actor_names.push(supervisable.name().to_string()); } + Health::Success => {} } } - if !failure_or_unhealthy_actors.is_empty() { - error!( - task_id=%self.task_id, - healthy_actors=?healthy_actors, - failed_or_unhealthy_actors=?failure_or_unhealthy_actors, - success_actors=?success_actors, - "compaction pipeline actor failure detected" - ); - return Health::FailureOrUnhealthy; + if !failure_actor_names.is_empty() { + let error_msg = format!("failed actors: {:?}", failure_actor_names); + error!(task_id=%self.task_id, "{error_msg}"); + self.status = PipelineStatus::Failed { error: error_msg }; + return; } - if healthy_actors.is_empty() { + if !has_healthy { debug!(task_id=%self.task_id, "all compaction pipeline actors completed"); - return Health::Success; - } - Health::Healthy - } - - pub async fn terminate(&mut self) { - self.kill_switch.kill(); - if let Some(handles) = self.handles.take() { - tokio::join!( - handles.merge_split_downloader.kill(), - handles.merge_executor.kill(), - handles.merge_packager.kill(), - handles.merge_uploader.kill(), - handles.merge_publisher.kill(), - ); + self.status = PipelineStatus::Completed; } } - /// Terminates the current actor chain, increments retry count, and - /// re-spawns. Downloaded splits remain on disk in the scratch directory. - pub async fn restart(&mut self, ctx: &ActorContext) { - self.terminate().await; - self.retry_count += 1; - if let Err(err) = self.spawn_pipeline(ctx) { - error!(task_id=%self.task_id, error=?err, "failed to respawn compaction pipeline"); + fn build_status_update(&self) -> PipelineStatusUpdate { + PipelineStatusUpdate { + task_id: self.task_id.clone(), + index_uid: self.pipeline_id.index_uid.clone(), + source_id: self.pipeline_id.source_id.clone(), + split_ids: self + .merge_operation + .splits_as_slice() + .iter() + .map(|split| split.split_id().to_string()) + .collect(), + merged_split_id: self.merge_operation.merge_split_id.clone(), + status: self.status.clone(), } } /// Spawns the 5-actor merge execution chain and sends the `MergeOperation` /// to the downloader to kick off execution. - fn spawn_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { - self.kill_switch = ctx.kill_switch().child(); - + pub(crate) fn spawn_pipeline(&mut self, spawn_ctx: &SpawnContext) -> anyhow::Result<()> { info!( task_id=%self.task_id, pipeline_id=%self.pipeline_id, @@ -206,9 +209,9 @@ impl CompactionPipeline { None, None, ); - let (merge_publisher_mailbox, merge_publisher_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) + let (merge_publisher_mailbox, merge_publisher_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) .spawn(merge_publisher); // Uploader @@ -222,17 +225,17 @@ impl CompactionPipeline { self.max_concurrent_split_uploads, self.event_broker.clone(), ); - let (merge_uploader_mailbox, merge_uploader_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) + let (merge_uploader_mailbox, merge_uploader_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) .spawn(merge_uploader); // Packager let tag_fields = self.doc_mapper.tag_named_fields()?; let merge_packager = Packager::new("MergePackager", tag_fields, merge_uploader_mailbox); - let (merge_packager_mailbox, merge_packager_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) + let (merge_packager_mailbox, merge_packager_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) .spawn(merge_packager); // MergeExecutor @@ -249,9 +252,9 @@ impl CompactionPipeline { merge_executor_io_controls, merge_packager_mailbox, ); - let (merge_executor_mailbox, merge_executor_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) + let (merge_executor_mailbox, merge_executor_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) .spawn(merge_executor); // MergeSplitDownloader @@ -261,9 +264,9 @@ impl CompactionPipeline { executor_mailbox: merge_executor_mailbox, io_controls: split_downloader_io_controls, }; - let (merge_split_downloader_mailbox, merge_split_downloader_handle) = ctx - .spawn_actor() - .set_kill_switch(self.kill_switch.clone()) + let (merge_split_downloader_mailbox, merge_split_downloader_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) .spawn(merge_split_downloader); // Kick off the pipeline. @@ -286,10 +289,10 @@ impl CompactionPipeline { } #[cfg(test)] -mod tests { +pub(crate) mod tests { use std::sync::Arc; - use quickwit_actors::Health; + use quickwit_actors::Universe; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_doc_mapper::default_doc_mapper_for_test; @@ -301,13 +304,16 @@ mod tests { use quickwit_proto::types::{IndexUid, NodeId}; use quickwit_storage::RamStorage; - use super::CompactionPipeline; + use super::{CompactionPipeline, PipelineStatus}; - fn test_pipeline() -> CompactionPipeline { + pub fn test_pipeline(task_id: &str, split_ids: &[&str]) -> CompactionPipeline { let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); - let splits = vec![SplitMetadata::for_test("split-1".to_string())]; + let splits: Vec = split_ids + .iter() + .map(|id| SplitMetadata::for_test(id.to_string())) + .collect(); let merge_operation = MergeOperation::new_merge_operation(splits); let pipeline_id = MergePipelineId { node_id: NodeId::from("test-node"), @@ -315,7 +321,7 @@ mod tests { source_id: "test-source".to_string(), }; CompactionPipeline::new( - "test-task".to_string(), + task_id.to_string(), TempDirectory::for_test(), merge_operation, pipeline_id, @@ -330,17 +336,51 @@ mod tests { ) } - #[test] - fn test_pipeline_no_handles_is_healthy() { - let pipeline = test_pipeline(); + #[tokio::test] + async fn test_spawn_pipeline_creates_handles() { + let universe = Universe::new(); + let mut pipeline = test_pipeline("task-1", &["split-1", "split-2"]); assert!(pipeline.handles.is_none()); - assert_eq!(pipeline.check_actor_health(), Health::Healthy); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + assert!(pipeline.handles.is_some()); + universe.assert_quit().await; } #[tokio::test] - async fn test_pipeline_terminate_without_handles() { - let mut pipeline = test_pipeline(); - pipeline.terminate().await; - assert!(pipeline.handles.is_none()); + async fn test_status_update_unspawned_pipeline() { + let mut pipeline = test_pipeline("task-1", &["split-1"]); + let update = pipeline.pipeline_status_update(); + assert_eq!(update.status, PipelineStatus::InProgress); + assert_eq!(update.task_id, "task-1"); + assert_eq!(update.split_ids, vec!["split-1"]); + assert_eq!(update.source_id, "test-source"); + assert_eq!(update.index_uid, IndexUid::for_test("test-index", 0)); + } + + #[tokio::test] + async fn test_status_update_healthy_pipeline() { + let universe = Universe::new(); + let mut pipeline = test_pipeline("task-1", &["split-1"]); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + let update = pipeline.pipeline_status_update(); + assert_eq!(update.status, PipelineStatus::InProgress); + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_killed_pipeline_fails() { + let universe = Universe::new(); + let mut pipeline = test_pipeline("task-1", &["split-1"]); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + + pipeline.kill_switch.kill(); + tokio::task::yield_now().await; + let update = pipeline.pipeline_status_update(); + assert!(matches!(update.status, PipelineStatus::Failed { .. })); + + // Calling again still returns Failed (sticky). + let update = pipeline.pipeline_status_update(); + assert!(matches!(update.status, PipelineStatus::Failed { .. })); + universe.assert_quit().await; } } diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index 4ba270c270b..e2809c4e8d3 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -15,36 +15,34 @@ use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Health}; +use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; use quickwit_common::io::Limiter; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_indexing::IndexingSplitStore; +use quickwit_proto::compaction::{ + CompactionFailure, CompactionInProgress, CompactionPlannerServiceClient, CompactionSuccess, + ReportStatusRequest, +}; use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_proto::types::NodeId; use quickwit_storage::StorageResolver; -use serde::Serialize; -use tracing::{error, info}; +use tracing::info; -use crate::compaction_pipeline::CompactionPipeline; +use crate::compaction_pipeline::{CompactionPipeline, PipelineStatus, PipelineStatusUpdate}; -const SUPERVISE_LOOP_INTERVAL: Duration = Duration::from_secs(1); +const CHECK_PIPELINE_STATUSES_INTERVAL: Duration = Duration::from_secs(1); #[derive(Debug)] -struct SuperviseLoop; - -#[derive(Clone, Debug, Default, Serialize)] -pub struct CompactorSupervisorState { - pub num_pipeline_slots: usize, - pub num_occupied_slots: usize, - pub num_completed_tasks: usize, - pub num_failed_tasks: usize, -} +struct CheckPipelineStatuses; /// Manages a pool of `CompactionPipeline`s, each executing a single merge task. /// -/// Periodically checks pipeline health, handles retries on failure, and reaps -/// completed/failed pipelines. +/// Periodically collects pipeline status updates and forwards them to the +/// compaction planner. Pipelines manage their own retry logic internally. pub struct CompactorSupervisor { + node_id: NodeId, + planner_client: CompactionPlannerServiceClient, pipelines: Vec>, // Shared resources distributed to pipelines when spawning actor chains. @@ -57,17 +55,13 @@ pub struct CompactorSupervisor { // Scratch directory root (/compaction/). compaction_root_directory: TempDirectory, - - max_local_retries: usize, - - // dummy counters until we have real state - num_completed_tasks: usize, - num_failed_tasks: usize, } impl CompactorSupervisor { #[allow(clippy::too_many_arguments)] pub fn new( + node_id: NodeId, + planner_client: CompactionPlannerServiceClient, num_pipeline_slots: usize, io_throughput_limiter: Option, split_store: IndexingSplitStore, @@ -76,10 +70,11 @@ impl CompactorSupervisor { max_concurrent_split_uploads: usize, event_broker: EventBroker, compaction_root_directory: TempDirectory, - max_local_retries: usize, ) -> Self { let pipelines = (0..num_pipeline_slots).map(|_| None).collect(); CompactorSupervisor { + node_id, + planner_client, pipelines, io_throughput_limiter, split_store, @@ -88,46 +83,66 @@ impl CompactorSupervisor { max_concurrent_split_uploads, event_broker, compaction_root_directory, - max_local_retries, - num_completed_tasks: 0, - num_failed_tasks: 0, } } - async fn supervise(&mut self, ctx: &ActorContext) { + fn check_pipeline_statuses(&mut self) -> Vec { + let mut statuses = Vec::new(); for slot in &mut self.pipelines { let Some(pipeline) = slot else { continue; }; + statuses.push(pipeline.pipeline_status_update()); + } + statuses + } + + fn build_report_status_request( + &self, + statuses: &[PipelineStatusUpdate], + ) -> ReportStatusRequest { + let in_progress_count = statuses + .iter() + .filter(|s| matches!(s.status, PipelineStatus::InProgress)) + .count(); + let available_slots = (self.pipelines.len() - in_progress_count) as u32; + + let mut in_progress = Vec::new(); + let mut successes = Vec::new(); + let mut failures = Vec::new(); - match pipeline.check_actor_health() { - Health::Healthy => {} - Health::Success => { - info!(task_id=%pipeline.task_id, "compaction task completed"); - self.num_completed_tasks += 1; - *slot = None; + for update in statuses { + match &update.status { + PipelineStatus::InProgress => { + in_progress.push(CompactionInProgress { + task_id: update.task_id.clone(), + index_uid: Some(update.index_uid.clone()), + source_id: update.source_id.clone(), + split_ids: update.split_ids.clone(), + }); } - Health::FailureOrUnhealthy => { - if pipeline.retry_count < self.max_local_retries { - info!( - task_id=%pipeline.task_id, - retry_count=%pipeline.retry_count, - "retrying compaction pipeline" - ); - pipeline.restart(ctx).await; - } else { - error!( - task_id=%pipeline.task_id, - retry_count=%pipeline.retry_count, - "compaction pipeline exhausted retries" - ); - pipeline.terminate().await; - self.num_failed_tasks += 1; - *slot = None; - } + PipelineStatus::Completed => { + successes.push(CompactionSuccess { + task_id: update.task_id.clone(), + merged_split_id: update.merged_split_id.clone(), + }); + } + PipelineStatus::Failed { error } => { + failures.push(CompactionFailure { + task_id: update.task_id.clone(), + error_message: error.clone(), + }); } } } + + ReportStatusRequest { + node_id: self.node_id.to_string(), + available_slots, + in_progress, + successes, + failures, + } } } @@ -146,22 +161,24 @@ impl Actor for CompactorSupervisor { num_pipeline_slots=%self.pipelines.len(), "compactor supervisor started" ); - ctx.schedule_self_msg(SUPERVISE_LOOP_INTERVAL, SuperviseLoop); + ctx.schedule_self_msg(CHECK_PIPELINE_STATUSES_INTERVAL, CheckPipelineStatuses); Ok(()) } } #[async_trait] -impl Handler for CompactorSupervisor { +impl Handler for CompactorSupervisor { type Reply = (); async fn handle( &mut self, - _msg: SuperviseLoop, + _msg: CheckPipelineStatuses, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - self.supervise(ctx).await; - ctx.schedule_self_msg(SUPERVISE_LOOP_INTERVAL, SuperviseLoop); + let statuses = self.check_pipeline_statuses(); + let _request = self.build_report_status_request(&statuses); + // TODO: send request to planner via gRPC, clear completed/failed slots on success. + ctx.schedule_self_msg(CHECK_PIPELINE_STATUSES_INTERVAL, CheckPipelineStatuses); Ok(()) } } @@ -172,16 +189,23 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::temp_dir::TempDirectory; + use quickwit_proto::compaction::CompactionPlannerServiceClient; use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; + use quickwit_proto::types::NodeId; use quickwit_storage::{RamStorage, StorageResolver}; use super::*; + use crate::compaction_pipeline::tests::test_pipeline; + use crate::planner::StubCompactionPlannerService; fn test_supervisor(num_slots: usize) -> CompactorSupervisor { let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); + let compaction_client = CompactionPlannerServiceClient::new(StubCompactionPlannerService); CompactorSupervisor::new( + NodeId::from("test-node"), + compaction_client, num_slots, None, split_store, @@ -190,30 +214,115 @@ mod tests { 2, EventBroker::default(), TempDirectory::for_test(), - 2, ) } + #[test] + fn test_check_pipeline_statuses_empty_slots() { + let mut supervisor = test_supervisor(4); + let statuses = supervisor.check_pipeline_statuses(); + assert!(statuses.is_empty()); + } + #[tokio::test] - async fn test_supervisor_starts_with_empty_slots() { - let universe = Universe::with_accelerated_time(); - let supervisor = test_supervisor(4); - let (_mailbox, handle) = universe.spawn_builder().spawn(supervisor); - let obs = handle.process_pending_and_observe().await; - assert_eq!(obs.obs_type, quickwit_actors::ObservationType::Alive); + async fn test_check_pipeline_statuses_with_pipelines() { + let universe = Universe::new(); + let mut supervisor = test_supervisor(4); + + let mut pipeline = test_pipeline("task-1", &["split-a", "split-b"]); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + supervisor.pipelines[0] = Some(pipeline); + + let mut pipeline = test_pipeline("task-2", &["split-c"]); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + supervisor.pipelines[2] = Some(pipeline); + + let statuses = supervisor.check_pipeline_statuses(); + assert_eq!(statuses.len(), 2); + assert_eq!(statuses[0].task_id, "task-1"); + assert_eq!(statuses[0].split_ids, vec!["split-a", "split-b"]); + assert_eq!(statuses[1].task_id, "task-2"); + assert_eq!(statuses[1].split_ids, vec!["split-c"]); universe.assert_quit().await; } #[tokio::test] - async fn test_supervisor_supervise_reaps_no_handle_pipelines() { - // A pipeline with no handles returns Healthy, so it stays in its slot. - // We spawn the supervisor as an actor so we can get a context for supervise(). - let universe = Universe::with_accelerated_time(); - let supervisor = test_supervisor(2); - let (_mailbox, handle) = universe.spawn_builder().spawn(supervisor); - // Let the supervisor run one supervision loop (it schedules SuperviseLoop on init). - let obs = handle.process_pending_and_observe().await; - assert_eq!(obs.obs_type, quickwit_actors::ObservationType::Alive); + async fn test_end_to_end_statuses_to_proto() { + let universe = Universe::new(); + let mut supervisor = test_supervisor(3); + + let mut pipeline = test_pipeline("task-1", &["s1", "s2"]); + pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); + supervisor.pipelines[0] = Some(pipeline); + + let statuses = supervisor.check_pipeline_statuses(); + let request = supervisor.build_report_status_request(&statuses); + + assert_eq!(request.node_id, "test-node"); + // 3 slots, 1 in-progress = 2 available + assert_eq!(request.available_slots, 2); + assert_eq!(request.in_progress.len(), 1); + assert_eq!(request.in_progress[0].task_id, "task-1"); + assert_eq!(request.in_progress[0].split_ids, vec!["s1", "s2"]); + assert!(request.successes.is_empty()); + assert!(request.failures.is_empty()); universe.assert_quit().await; } + + #[test] + fn test_build_report_status_request_empty() { + let supervisor = test_supervisor(4); + let request = supervisor.build_report_status_request(&[]); + assert_eq!(request.node_id, "test-node"); + assert_eq!(request.available_slots, 4); + assert!(request.in_progress.is_empty()); + assert!(request.successes.is_empty()); + assert!(request.failures.is_empty()); + } + + #[test] + fn test_build_report_status_request_mixed_statuses() { + let supervisor = test_supervisor(4); + let statuses = vec![ + PipelineStatusUpdate { + task_id: "task-1".to_string(), + index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), + source_id: "src".to_string(), + split_ids: vec!["s1".to_string(), "s2".to_string()], + merged_split_id: "merged-1".to_string(), + status: PipelineStatus::InProgress, + }, + PipelineStatusUpdate { + task_id: "task-2".to_string(), + index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), + source_id: "src".to_string(), + split_ids: vec!["s3".to_string()], + merged_split_id: "merged-2".to_string(), + status: PipelineStatus::Completed, + }, + PipelineStatusUpdate { + task_id: "task-3".to_string(), + index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), + source_id: "src".to_string(), + split_ids: vec!["s4".to_string()], + merged_split_id: "merged-3".to_string(), + status: PipelineStatus::Failed { + error: "boom".to_string(), + }, + }, + ]; + + let request = supervisor.build_report_status_request(&statuses); + + assert_eq!(request.available_slots, 3); + assert_eq!(request.in_progress.len(), 1); + assert_eq!(request.in_progress[0].task_id, "task-1"); + assert_eq!(request.in_progress[0].split_ids, vec!["s1", "s2"]); + assert_eq!(request.successes.len(), 1); + assert_eq!(request.successes[0].task_id, "task-2"); + assert_eq!(request.successes[0].merged_split_id, "merged-2"); + assert_eq!(request.failures.len(), 1); + assert_eq!(request.failures[0].task_id, "task-3"); + assert_eq!(request.failures[0].error_message, "boom"); + } } diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs index fd2c5b05885..05f79d83f73 100644 --- a/quickwit/quickwit-compaction/src/lib.rs +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -27,12 +27,17 @@ use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_config::CompactorConfig; use quickwit_indexing::IndexingSplitStore; +use quickwit_proto::compaction::CompactionPlannerServiceClient; use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_proto::types::NodeId; use quickwit_storage::StorageResolver; use tracing::info; +#[allow(clippy::too_many_arguments)] pub async fn start_compactor_service( universe: &Universe, + node_id: NodeId, + compaction_client: CompactionPlannerServiceClient, compactor_config: &CompactorConfig, split_store: IndexingSplitStore, metastore: MetastoreServiceClient, @@ -41,8 +46,11 @@ pub async fn start_compactor_service( compaction_root_directory: TempDirectory, ) -> anyhow::Result> { info!("starting compactor service"); + // TODO: configure this for real let io_throughput_limiter = compactor_config.max_merge_write_throughput.map(io::limiter); let supervisor = CompactorSupervisor::new( + node_id, + compaction_client, compactor_config.max_concurrent_pipelines.get(), io_throughput_limiter, split_store, @@ -51,7 +59,6 @@ pub async fn start_compactor_service( compactor_config.max_concurrent_split_uploads, event_broker, compaction_root_directory, - compactor_config.max_local_retries, ); let (mailbox, _handle) = universe.spawn_builder().spawn(supervisor); Ok(mailbox) diff --git a/quickwit/quickwit-compaction/src/planner/compaction_service.rs b/quickwit/quickwit-compaction/src/planner/compaction_service.rs index 0e3104a2129..340ff4e0d5c 100644 --- a/quickwit/quickwit-compaction/src/planner/compaction_service.rs +++ b/quickwit/quickwit-compaction/src/planner/compaction_service.rs @@ -13,14 +13,24 @@ // limitations under the License. use async_trait::async_trait; -use quickwit_proto::compaction::{CompactionResult, CompactionService, PingRequest, PingResponse}; +use quickwit_proto::compaction::{ + CompactionPlannerService, CompactionResult, PingRequest, PingResponse, ReportStatusRequest, + ReportStatusResponse, +}; #[derive(Debug, Clone)] -pub struct StubCompactionService; +pub struct StubCompactionPlannerService; #[async_trait] -impl CompactionService for StubCompactionService { +impl CompactionPlannerService for StubCompactionPlannerService { async fn ping(&self, _request: PingRequest) -> CompactionResult { Ok(PingResponse {}) } + + async fn report_status( + &self, + _request: ReportStatusRequest, + ) -> CompactionResult { + Ok(ReportStatusResponse {}) + } } diff --git a/quickwit/quickwit-compaction/src/planner/mod.rs b/quickwit/quickwit-compaction/src/planner/mod.rs index 3b531a439f2..5ccd750a20c 100644 --- a/quickwit/quickwit-compaction/src/planner/mod.rs +++ b/quickwit/quickwit-compaction/src/planner/mod.rs @@ -14,4 +14,4 @@ mod compaction_service; -pub use compaction_service::StubCompactionService; +pub use compaction_service::StubCompactionPlannerService; diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 47c3b6a77cb..ff981fcf31c 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -256,9 +256,6 @@ pub struct CompactorConfig { /// Limits the IO throughput of the split downloader and the merge executor. #[serde(default)] pub max_merge_write_throughput: Option, - /// Maximum local retries before reporting a merge as failed to the planner. - #[serde(default = "CompactorConfig::default_max_local_retries")] - pub max_local_retries: usize, /// Maximum size of the local split store cache in bytes. #[serde(default = "CompactorConfig::default_split_store_max_num_bytes")] pub split_store_max_num_bytes: ByteSize, @@ -276,10 +273,6 @@ impl CompactorConfig { 12 } - fn default_max_local_retries() -> usize { - 2 - } - pub fn default_split_store_max_num_bytes() -> ByteSize { ByteSize::gib(100) } @@ -294,7 +287,6 @@ impl CompactorConfig { max_concurrent_pipelines: NonZeroUsize::new(2).unwrap(), max_concurrent_split_uploads: 4, max_merge_write_throughput: None, - max_local_retries: 2, split_store_max_num_bytes: ByteSize::mb(1), split_store_max_num_splits: 3, } @@ -307,7 +299,6 @@ impl Default for CompactorConfig { max_concurrent_pipelines: Self::default_max_concurrent_pipelines(), max_concurrent_split_uploads: Self::default_max_concurrent_split_uploads(), max_merge_write_throughput: None, - max_local_retries: Self::default_max_local_retries(), split_store_max_num_bytes: Self::default_split_store_max_num_bytes(), split_store_max_num_splits: Self::default_split_store_max_num_splits(), } diff --git a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs index 7b425061134..9c2fe3effde 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs @@ -683,7 +683,11 @@ impl ClusterSandbox { #[tokio::test] async fn test_sandbox_happy_path() { let sandbox = ClusterSandboxBuilder::default() - .add_node([QuickwitService::ControlPlane, QuickwitService::Metastore]) + .add_node([ + QuickwitService::ControlPlane, + QuickwitService::Metastore, + QuickwitService::Janitor, + ]) .add_node([QuickwitService::Searcher]) .add_node([QuickwitService::Indexer]) .build_and_start() @@ -696,7 +700,11 @@ async fn test_sandbox_happy_path() { #[tokio::test] async fn test_sandbox_add_node_dynamically() { let mut sandbox = ClusterSandboxBuilder::default() - .add_node([QuickwitService::ControlPlane, QuickwitService::Metastore]) + .add_node([ + QuickwitService::ControlPlane, + QuickwitService::Metastore, + QuickwitService::Janitor, + ]) .add_node([QuickwitService::Searcher]) .build_and_start() .await; diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index a83e8723bd2..7671fd9ea19 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -38,10 +38,12 @@ fn main() -> Result<(), Box> { // Compaction service. let mut prost_config = prost_build::Config::default(); prost_config.file_descriptor_set_path("src/codegen/quickwit/compaction_descriptor.bin"); + prost_config.extern_path(".quickwit.common.IndexUid", "crate::types::IndexUid"); Codegen::builder() .with_prost_config(prost_config) .with_protos(&["protos/quickwit/compaction.proto"]) + .with_includes(&["protos"]) .with_output_dir("src/codegen/quickwit") .with_result_type_path("crate::compaction::CompactionResult") .with_error_type_path("crate::compaction::CompactionError") diff --git a/quickwit/quickwit-proto/protos/quickwit/compaction.proto b/quickwit/quickwit-proto/protos/quickwit/compaction.proto index 82979c3b4ee..f071cc3bc01 100644 --- a/quickwit/quickwit-proto/protos/quickwit/compaction.proto +++ b/quickwit/quickwit-proto/protos/quickwit/compaction.proto @@ -16,9 +16,39 @@ syntax = "proto3"; package quickwit.compaction; -service CompactionService { +import "quickwit/common.proto"; + +service CompactionPlannerService { rpc Ping(PingRequest) returns (PingResponse); + rpc ReportStatus(ReportStatusRequest) returns (ReportStatusResponse); } message PingRequest {} -message PingResponse {} \ No newline at end of file +message PingResponse {} + +message ReportStatusRequest { + string node_id = 1; + uint32 available_slots = 2; + repeated CompactionInProgress in_progress = 3; + repeated CompactionSuccess successes = 4; + repeated CompactionFailure failures = 5; +} + +message CompactionInProgress { + string task_id = 1; + quickwit.common.IndexUid index_uid = 2; + string source_id = 3; + repeated string split_ids = 4; +} + +message CompactionSuccess { + string task_id = 1; + string merged_split_id = 2; +} + +message CompactionFailure { + string task_id = 1; + string error_message = 2; +} + +message ReportStatusResponse {} \ No newline at end of file diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs index eec6be161ca..df33d068f9a 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs @@ -5,6 +5,51 @@ pub struct PingRequest {} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] pub struct PingResponse {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportStatusRequest { + #[prost(string, tag = "1")] + pub node_id: ::prost::alloc::string::String, + #[prost(uint32, tag = "2")] + pub available_slots: u32, + #[prost(message, repeated, tag = "3")] + pub in_progress: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "4")] + pub successes: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "5")] + pub failures: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct CompactionInProgress { + #[prost(string, tag = "1")] + pub task_id: ::prost::alloc::string::String, + #[prost(message, optional, tag = "2")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "3")] + pub source_id: ::prost::alloc::string::String, + #[prost(string, repeated, tag = "4")] + pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct CompactionSuccess { + #[prost(string, tag = "1")] + pub task_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub merged_split_id: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct CompactionFailure { + #[prost(string, tag = "1")] + pub task_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub error_message: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] +pub struct ReportStatusResponse {} /// BEGIN quickwit-codegen #[allow(unused_imports)] use std::str::FromStr; @@ -15,43 +60,54 @@ impl RpcName for PingRequest { "ping" } } +impl RpcName for ReportStatusRequest { + fn rpc_name() -> &'static str { + "report_status" + } +} #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait CompactionService: std::fmt::Debug + Send + Sync + 'static { +pub trait CompactionPlannerService: std::fmt::Debug + Send + Sync + 'static { async fn ping( &self, request: PingRequest, ) -> crate::compaction::CompactionResult; + async fn report_status( + &self, + request: ReportStatusRequest, + ) -> crate::compaction::CompactionResult; } #[derive(Debug, Clone)] -pub struct CompactionServiceClient { - inner: InnerCompactionServiceClient, +pub struct CompactionPlannerServiceClient { + inner: InnerCompactionPlannerServiceClient, } #[derive(Debug, Clone)] -struct InnerCompactionServiceClient(std::sync::Arc); -impl CompactionServiceClient { +struct InnerCompactionPlannerServiceClient(std::sync::Arc); +impl CompactionPlannerServiceClient { pub fn new(instance: T) -> Self where - T: CompactionService, + T: CompactionPlannerService, { #[cfg(any(test, feature = "testsuite"))] assert!( std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < - MockCompactionService > (), - "`MockCompactionService` must be wrapped in a `MockCompactionServiceWrapper`: use `CompactionServiceClient::from_mock(mock)` to instantiate the client" + MockCompactionPlannerService > (), + "`MockCompactionPlannerService` must be wrapped in a `MockCompactionPlannerServiceWrapper`: use `CompactionPlannerServiceClient::from_mock(mock)` to instantiate the client" ); Self { - inner: InnerCompactionServiceClient(std::sync::Arc::new(instance)), + inner: InnerCompactionPlannerServiceClient(std::sync::Arc::new(instance)), } } pub fn as_grpc_service( &self, max_message_size: bytesize::ByteSize, - ) -> compaction_service_grpc_server::CompactionServiceGrpcServer< - CompactionServiceGrpcServerAdapter, + ) -> compaction_planner_service_grpc_server::CompactionPlannerServiceGrpcServer< + CompactionPlannerServiceGrpcServerAdapter, > { - let adapter = CompactionServiceGrpcServerAdapter::new(self.clone()); - compaction_service_grpc_server::CompactionServiceGrpcServer::new(adapter) + let adapter = CompactionPlannerServiceGrpcServerAdapter::new(self.clone()); + compaction_planner_service_grpc_server::CompactionPlannerServiceGrpcServer::new( + adapter, + ) .accept_compressed(tonic::codec::CompressionEncoding::Gzip) .accept_compressed(tonic::codec::CompressionEncoding::Zstd) .send_compressed(tonic::codec::CompressionEncoding::Gzip) @@ -68,7 +124,7 @@ impl CompactionServiceClient { let (_, connection_keys_watcher) = tokio::sync::watch::channel( std::collections::HashSet::from_iter([addr]), ); - let mut client = compaction_service_grpc_client::CompactionServiceGrpcClient::new( + let mut client = compaction_planner_service_grpc_client::CompactionPlannerServiceGrpcClient::new( channel, ) .max_decoding_message_size(max_message_size.0 as usize) @@ -78,7 +134,7 @@ impl CompactionServiceClient { .accept_compressed(compression_encoding) .send_compressed(compression_encoding); } - let adapter = CompactionServiceGrpcClientAdapter::new( + let adapter = CompactionPlannerServiceGrpcClientAdapter::new( client, connection_keys_watcher, ); @@ -88,9 +144,9 @@ impl CompactionServiceClient { balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, compression_encoding_opt: Option, - ) -> CompactionServiceClient { + ) -> CompactionPlannerServiceClient { let connection_keys_watcher = balance_channel.connection_keys_watcher(); - let mut client = compaction_service_grpc_client::CompactionServiceGrpcClient::new( + let mut client = compaction_planner_service_grpc_client::CompactionPlannerServiceGrpcClient::new( balance_channel, ) .max_decoding_message_size(max_message_size.0 as usize) @@ -100,7 +156,7 @@ impl CompactionServiceClient { .accept_compressed(compression_encoding) .send_compressed(compression_encoding); } - let adapter = CompactionServiceGrpcClientAdapter::new( + let adapter = CompactionPlannerServiceGrpcClientAdapter::new( client, connection_keys_watcher, ); @@ -109,55 +165,69 @@ impl CompactionServiceClient { pub fn from_mailbox(mailbox: quickwit_actors::Mailbox) -> Self where A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, - CompactionServiceMailbox: CompactionService, + CompactionPlannerServiceMailbox: CompactionPlannerService, { - CompactionServiceClient::new(CompactionServiceMailbox::new(mailbox)) + CompactionPlannerServiceClient::new( + CompactionPlannerServiceMailbox::new(mailbox), + ) } - pub fn tower() -> CompactionServiceTowerLayerStack { - CompactionServiceTowerLayerStack::default() + pub fn tower() -> CompactionPlannerServiceTowerLayerStack { + CompactionPlannerServiceTowerLayerStack::default() } #[cfg(any(test, feature = "testsuite"))] - pub fn from_mock(mock: MockCompactionService) -> Self { - let mock_wrapper = mock_compaction_service::MockCompactionServiceWrapper { + pub fn from_mock(mock: MockCompactionPlannerService) -> Self { + let mock_wrapper = mock_compaction_planner_service::MockCompactionPlannerServiceWrapper { inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } #[cfg(any(test, feature = "testsuite"))] pub fn mocked() -> Self { - Self::from_mock(MockCompactionService::new()) + Self::from_mock(MockCompactionPlannerService::new()) } } #[async_trait::async_trait] -impl CompactionService for CompactionServiceClient { +impl CompactionPlannerService for CompactionPlannerServiceClient { async fn ping( &self, request: PingRequest, ) -> crate::compaction::CompactionResult { self.inner.0.ping(request).await } + async fn report_status( + &self, + request: ReportStatusRequest, + ) -> crate::compaction::CompactionResult { + self.inner.0.report_status(request).await + } } #[cfg(any(test, feature = "testsuite"))] -pub mod mock_compaction_service { +pub mod mock_compaction_planner_service { use super::*; #[derive(Debug)] - pub struct MockCompactionServiceWrapper { - pub(super) inner: tokio::sync::Mutex, + pub struct MockCompactionPlannerServiceWrapper { + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] - impl CompactionService for MockCompactionServiceWrapper { + impl CompactionPlannerService for MockCompactionPlannerServiceWrapper { async fn ping( &self, request: super::PingRequest, ) -> crate::compaction::CompactionResult { self.inner.lock().await.ping(request).await } + async fn report_status( + &self, + request: super::ReportStatusRequest, + ) -> crate::compaction::CompactionResult { + self.inner.lock().await.report_status(request).await + } } } pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for InnerCompactionServiceClient { +impl tower::Service for InnerCompactionPlannerServiceClient { type Response = PingResponse; type Error = crate::compaction::CompactionError; type Future = BoxFuture; @@ -173,25 +243,52 @@ impl tower::Service for InnerCompactionServiceClient { Box::pin(fut) } } +impl tower::Service for InnerCompactionPlannerServiceClient { + type Response = ReportStatusResponse; + type Error = crate::compaction::CompactionError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ReportStatusRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.report_status(request).await }; + Box::pin(fut) + } +} /// A tower service stack is a set of tower services. #[derive(Debug)] -struct CompactionServiceTowerServiceStack { +struct CompactionPlannerServiceTowerServiceStack { #[allow(dead_code)] - inner: InnerCompactionServiceClient, + inner: InnerCompactionPlannerServiceClient, ping_svc: quickwit_common::tower::BoxService< PingRequest, PingResponse, crate::compaction::CompactionError, >, + report_status_svc: quickwit_common::tower::BoxService< + ReportStatusRequest, + ReportStatusResponse, + crate::compaction::CompactionError, + >, } #[async_trait::async_trait] -impl CompactionService for CompactionServiceTowerServiceStack { +impl CompactionPlannerService for CompactionPlannerServiceTowerServiceStack { async fn ping( &self, request: PingRequest, ) -> crate::compaction::CompactionResult { self.ping_svc.clone().ready().await?.call(request).await } + async fn report_status( + &self, + request: ReportStatusRequest, + ) -> crate::compaction::CompactionResult { + self.report_status_svc.clone().ready().await?.call(request).await + } } type PingLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< @@ -203,11 +300,22 @@ type PingLayer = quickwit_common::tower::BoxLayer< PingResponse, crate::compaction::CompactionError, >; +type ReportStatusLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + ReportStatusRequest, + ReportStatusResponse, + crate::compaction::CompactionError, + >, + ReportStatusRequest, + ReportStatusResponse, + crate::compaction::CompactionError, +>; #[derive(Debug, Default)] -pub struct CompactionServiceTowerLayerStack { +pub struct CompactionPlannerServiceTowerLayerStack { ping_layers: Vec, + report_status_layers: Vec, } -impl CompactionServiceTowerLayerStack { +impl CompactionPlannerServiceTowerLayerStack { pub fn stack_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -235,8 +343,35 @@ impl CompactionServiceTowerLayerStack { crate::compaction::CompactionError, >, >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + ReportStatusRequest, + ReportStatusResponse, + crate::compaction::CompactionError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + ReportStatusRequest, + Response = ReportStatusResponse, + Error = crate::compaction::CompactionError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, { self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.report_status_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self } pub fn stack_ping_layer(mut self, layer: L) -> Self @@ -258,11 +393,32 @@ impl CompactionServiceTowerLayerStack { self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer)); self } - pub fn build(self, instance: T) -> CompactionServiceClient + pub fn stack_report_status_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + ReportStatusRequest, + ReportStatusResponse, + crate::compaction::CompactionError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + ReportStatusRequest, + Response = ReportStatusResponse, + Error = crate::compaction::CompactionError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.report_status_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn build(self, instance: T) -> CompactionPlannerServiceClient where - T: CompactionService, + T: CompactionPlannerService, { - let inner_client = InnerCompactionServiceClient(std::sync::Arc::new(instance)); + let inner_client = InnerCompactionPlannerServiceClient( + std::sync::Arc::new(instance), + ); self.build_from_inner_client(inner_client) } pub fn build_from_channel( @@ -271,8 +427,8 @@ impl CompactionServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, compression_encoding_opt: Option, - ) -> CompactionServiceClient { - let client = CompactionServiceClient::from_channel( + ) -> CompactionPlannerServiceClient { + let client = CompactionPlannerServiceClient::from_channel( addr, channel, max_message_size, @@ -286,8 +442,8 @@ impl CompactionServiceTowerLayerStack { balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, compression_encoding_opt: Option, - ) -> CompactionServiceClient { - let client = CompactionServiceClient::from_balance_channel( + ) -> CompactionPlannerServiceClient { + let client = CompactionPlannerServiceClient::from_balance_channel( balance_channel, max_message_size, compression_encoding_opt, @@ -298,29 +454,29 @@ impl CompactionServiceTowerLayerStack { pub fn build_from_mailbox( self, mailbox: quickwit_actors::Mailbox, - ) -> CompactionServiceClient + ) -> CompactionPlannerServiceClient where A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, - CompactionServiceMailbox: CompactionService, + CompactionPlannerServiceMailbox: CompactionPlannerService, { - let inner_client = InnerCompactionServiceClient( - std::sync::Arc::new(CompactionServiceMailbox::new(mailbox)), + let inner_client = InnerCompactionPlannerServiceClient( + std::sync::Arc::new(CompactionPlannerServiceMailbox::new(mailbox)), ); self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock( self, - mock: MockCompactionService, - ) -> CompactionServiceClient { - let client = CompactionServiceClient::from_mock(mock); + mock: MockCompactionPlannerService, + ) -> CompactionPlannerServiceClient { + let client = CompactionPlannerServiceClient::from_mock(mock); let inner_client = client.inner; self.build_from_inner_client(inner_client) } fn build_from_inner_client( self, - inner_client: InnerCompactionServiceClient, - ) -> CompactionServiceClient { + inner_client: InnerCompactionPlannerServiceClient, + ) -> CompactionPlannerServiceClient { let ping_svc = self .ping_layers .into_iter() @@ -329,11 +485,20 @@ impl CompactionServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); - let tower_svc_stack = CompactionServiceTowerServiceStack { + let report_status_svc = self + .report_status_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let tower_svc_stack = CompactionPlannerServiceTowerServiceStack { inner: inner_client, ping_svc, + report_status_svc, }; - CompactionServiceClient::new(tower_svc_stack) + CompactionPlannerServiceClient::new(tower_svc_stack) } } #[derive(Debug, Clone)] @@ -351,10 +516,10 @@ where } } #[derive(Debug)] -pub struct CompactionServiceMailbox { +pub struct CompactionPlannerServiceMailbox { inner: MailboxAdapter, } -impl CompactionServiceMailbox { +impl CompactionPlannerServiceMailbox { pub fn new(instance: quickwit_actors::Mailbox) -> Self { let inner = MailboxAdapter { inner: instance, @@ -363,7 +528,7 @@ impl CompactionServiceMailbox { Self { inner } } } -impl Clone for CompactionServiceMailbox { +impl Clone for CompactionPlannerServiceMailbox { fn clone(&self) -> Self { let inner = MailboxAdapter { inner: self.inner.clone(), @@ -372,7 +537,7 @@ impl Clone for CompactionServiceMailbox { Self { inner } } } -impl tower::Service for CompactionServiceMailbox +impl tower::Service for CompactionPlannerServiceMailbox where A: quickwit_actors::Actor + quickwit_actors::DeferableReplyHandler> + Send @@ -403,17 +568,23 @@ where } } #[async_trait::async_trait] -impl CompactionService for CompactionServiceMailbox +impl CompactionPlannerService for CompactionPlannerServiceMailbox where A: quickwit_actors::Actor + std::fmt::Debug, - CompactionServiceMailbox< + CompactionPlannerServiceMailbox< A, >: tower::Service< - PingRequest, - Response = PingResponse, - Error = crate::compaction::CompactionError, - Future = BoxFuture, - >, + PingRequest, + Response = PingResponse, + Error = crate::compaction::CompactionError, + Future = BoxFuture, + > + + tower::Service< + ReportStatusRequest, + Response = ReportStatusResponse, + Error = crate::compaction::CompactionError, + Future = BoxFuture, + >, { async fn ping( &self, @@ -421,16 +592,22 @@ where ) -> crate::compaction::CompactionResult { self.clone().call(request).await } + async fn report_status( + &self, + request: ReportStatusRequest, + ) -> crate::compaction::CompactionResult { + self.clone().call(request).await + } } #[derive(Debug, Clone)] -pub struct CompactionServiceGrpcClientAdapter { +pub struct CompactionPlannerServiceGrpcClientAdapter { inner: T, #[allow(dead_code)] connection_addrs_rx: tokio::sync::watch::Receiver< std::collections::HashSet, >, } -impl CompactionServiceGrpcClientAdapter { +impl CompactionPlannerServiceGrpcClientAdapter { pub fn new( instance: T, connection_addrs_rx: tokio::sync::watch::Receiver< @@ -444,9 +621,9 @@ impl CompactionServiceGrpcClientAdapter { } } #[async_trait::async_trait] -impl CompactionService -for CompactionServiceGrpcClientAdapter< - compaction_service_grpc_client::CompactionServiceGrpcClient, +impl CompactionPlannerService +for CompactionPlannerServiceGrpcClientAdapter< + compaction_planner_service_grpc_client::CompactionPlannerServiceGrpcClient, > where T: tonic::client::GrpcService + std::fmt::Debug + Clone + Send @@ -470,24 +647,38 @@ where PingRequest::rpc_name(), )) } + async fn report_status( + &self, + request: ReportStatusRequest, + ) -> crate::compaction::CompactionResult { + self.inner + .clone() + .report_status(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + ReportStatusRequest::rpc_name(), + )) + } } #[derive(Debug)] -pub struct CompactionServiceGrpcServerAdapter { - inner: InnerCompactionServiceClient, +pub struct CompactionPlannerServiceGrpcServerAdapter { + inner: InnerCompactionPlannerServiceClient, } -impl CompactionServiceGrpcServerAdapter { +impl CompactionPlannerServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where - T: CompactionService, + T: CompactionPlannerService, { Self { - inner: InnerCompactionServiceClient(std::sync::Arc::new(instance)), + inner: InnerCompactionPlannerServiceClient(std::sync::Arc::new(instance)), } } } #[async_trait::async_trait] -impl compaction_service_grpc_server::CompactionServiceGrpc -for CompactionServiceGrpcServerAdapter { +impl compaction_planner_service_grpc_server::CompactionPlannerServiceGrpc +for CompactionPlannerServiceGrpcServerAdapter { async fn ping( &self, request: tonic::Request, @@ -499,9 +690,20 @@ for CompactionServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn report_status( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .report_status(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } } /// Generated client implementations. -pub mod compaction_service_grpc_client { +pub mod compaction_planner_service_grpc_client { #![allow( unused_variables, dead_code, @@ -512,10 +714,10 @@ pub mod compaction_service_grpc_client { use tonic::codegen::*; use tonic::codegen::http::Uri; #[derive(Debug, Clone)] - pub struct CompactionServiceGrpcClient { + pub struct CompactionPlannerServiceGrpcClient { inner: tonic::client::Grpc, } - impl CompactionServiceGrpcClient { + impl CompactionPlannerServiceGrpcClient { /// Attempt to create a new client by connecting to a given endpoint. pub async fn connect(dst: D) -> Result where @@ -526,7 +728,7 @@ pub mod compaction_service_grpc_client { Ok(Self::new(conn)) } } - impl CompactionServiceGrpcClient + impl CompactionPlannerServiceGrpcClient where T: tonic::client::GrpcService, T::Error: Into, @@ -544,7 +746,7 @@ pub mod compaction_service_grpc_client { pub fn with_interceptor( inner: T, interceptor: F, - ) -> CompactionServiceGrpcClient> + ) -> CompactionPlannerServiceGrpcClient> where F: tonic::service::Interceptor, T::ResponseBody: Default, @@ -558,7 +760,9 @@ pub mod compaction_service_grpc_client { http::Request, >>::Error: Into + std::marker::Send + std::marker::Sync, { - CompactionServiceGrpcClient::new(InterceptedService::new(inner, interceptor)) + CompactionPlannerServiceGrpcClient::new( + InterceptedService::new(inner, interceptor), + ) } /// Compress requests with the given encoding. /// @@ -605,19 +809,51 @@ pub mod compaction_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.compaction.CompactionService/Ping", + "/quickwit.compaction.CompactionPlannerService/Ping", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.compaction.CompactionPlannerService", + "Ping", + ), + ); + self.inner.unary(req, path, codec).await + } + pub async fn report_status( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::unknown( + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic_prost::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.compaction.CompactionPlannerService/ReportStatus", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new("quickwit.compaction.CompactionService", "Ping"), + GrpcMethod::new( + "quickwit.compaction.CompactionPlannerService", + "ReportStatus", + ), ); self.inner.unary(req, path, codec).await } } } /// Generated server implementations. -pub mod compaction_service_grpc_server { +pub mod compaction_planner_service_grpc_server { #![allow( unused_variables, dead_code, @@ -626,23 +862,30 @@ pub mod compaction_service_grpc_server { clippy::let_unit_value, )] use tonic::codegen::*; - /// Generated trait containing gRPC methods that should be implemented for use with CompactionServiceGrpcServer. + /// Generated trait containing gRPC methods that should be implemented for use with CompactionPlannerServiceGrpcServer. #[async_trait] - pub trait CompactionServiceGrpc: std::marker::Send + std::marker::Sync + 'static { + pub trait CompactionPlannerServiceGrpc: std::marker::Send + std::marker::Sync + 'static { async fn ping( &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; + async fn report_status( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; } #[derive(Debug)] - pub struct CompactionServiceGrpcServer { + pub struct CompactionPlannerServiceGrpcServer { inner: Arc, accept_compression_encodings: EnabledCompressionEncodings, send_compression_encodings: EnabledCompressionEncodings, max_decoding_message_size: Option, max_encoding_message_size: Option, } - impl CompactionServiceGrpcServer { + impl CompactionPlannerServiceGrpcServer { pub fn new(inner: T) -> Self { Self::from_arc(Arc::new(inner)) } @@ -694,9 +937,9 @@ pub mod compaction_service_grpc_server { } } impl tonic::codegen::Service> - for CompactionServiceGrpcServer + for CompactionPlannerServiceGrpcServer where - T: CompactionServiceGrpc, + T: CompactionPlannerServiceGrpc, B: Body + std::marker::Send + 'static, B::Error: Into + std::marker::Send + 'static, { @@ -711,11 +954,11 @@ pub mod compaction_service_grpc_server { } fn call(&mut self, req: http::Request) -> Self::Future { match req.uri().path() { - "/quickwit.compaction.CompactionService/Ping" => { + "/quickwit.compaction.CompactionPlannerService/Ping" => { #[allow(non_camel_case_types)] - struct PingSvc(pub Arc); + struct PingSvc(pub Arc); impl< - T: CompactionServiceGrpc, + T: CompactionPlannerServiceGrpc, > tonic::server::UnaryService for PingSvc { type Response = super::PingResponse; type Future = BoxFuture< @@ -728,7 +971,8 @@ pub mod compaction_service_grpc_server { ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { - ::ping(&inner, request).await + ::ping(&inner, request) + .await }; Box::pin(fut) } @@ -755,6 +999,55 @@ pub mod compaction_service_grpc_server { }; Box::pin(fut) } + "/quickwit.compaction.CompactionPlannerService/ReportStatus" => { + #[allow(non_camel_case_types)] + struct ReportStatusSvc(pub Arc); + impl< + T: CompactionPlannerServiceGrpc, + > tonic::server::UnaryService + for ReportStatusSvc { + type Response = super::ReportStatusResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::report_status( + &inner, + request, + ) + .await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let method = ReportStatusSvc(inner); + let codec = tonic_prost::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } _ => { Box::pin(async move { let mut response = http::Response::new( @@ -777,7 +1070,7 @@ pub mod compaction_service_grpc_server { } } } - impl Clone for CompactionServiceGrpcServer { + impl Clone for CompactionPlannerServiceGrpcServer { fn clone(&self) -> Self { let inner = self.inner.clone(); Self { @@ -790,8 +1083,8 @@ pub mod compaction_service_grpc_server { } } /// Generated gRPC service name - pub const SERVICE_NAME: &str = "quickwit.compaction.CompactionService"; - impl tonic::server::NamedService for CompactionServiceGrpcServer { + pub const SERVICE_NAME: &str = "quickwit.compaction.CompactionPlannerService"; + impl tonic::server::NamedService for CompactionPlannerServiceGrpcServer { const NAME: &'static str = SERVICE_NAME; } } diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index bf0d147b685..281bc73e7df 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -138,6 +138,7 @@ impl Display for IndexingPipelineId { /// Uniquely identifies a merge pipeline. There exists at most one merge pipeline per /// `(index_uid, source_id)` running on indexer at any given time fed by one or more indexing /// pipelines. +/// TODO: Rework/remove this as part of splitting up merges. #[derive(Clone, Debug, Hash, Eq, PartialEq)] pub struct MergePipelineId { pub node_id: NodeId, diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 8c974a4c990..3eda56d5e5e 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -72,7 +72,7 @@ use quickwit_common::tower::{ }; use quickwit_common::uri::Uri; use quickwit_common::{get_bool_from_env, spawn_named_task}; -use quickwit_compaction::planner::StubCompactionService; +use quickwit_compaction::planner::StubCompactionPlannerService; use quickwit_compaction::{CompactorSupervisor, start_compactor_service}; use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IngestApiConfig, NodeConfig}; @@ -95,7 +95,7 @@ use quickwit_metastore::{ ControlPlaneMetastore, ListIndexesMetadataResponseExt, MetastoreResolver, }; use quickwit_opentelemetry::otlp::{OtlpGrpcLogsService, OtlpGrpcTracesService}; -use quickwit_proto::compaction::CompactionServiceClient; +use quickwit_proto::compaction::CompactionPlannerServiceClient; use quickwit_proto::control_plane::ControlPlaneServiceClient; use quickwit_proto::indexing::{IndexingServiceClient, ShardPositionsUpdate}; use quickwit_proto::ingest::ingester::{ @@ -203,7 +203,7 @@ struct QuickwitServices { pub ingest_router_service: IngestRouterServiceClient, ingester_opt: Option, - pub compaction_service_client_opt: Option, + pub compaction_service_client_opt: Option, pub _compactor_supervisor_opt: Option>, pub janitor_service_opt: Option>, pub jaeger_service_opt: Option, @@ -271,28 +271,22 @@ async fn balance_channel_for_service( BalanceChannel::from_stream(service_change_stream) } -/// Builds a `CompactionServiceClient` if the compaction service is available. +/// Builds a `CompactionPlannerServiceClient` if the node runs the compactor. /// -/// On janitor nodes with `QW_ENABLE_COMPACTION_SERVICE=true`, wraps a local stub. -/// On non-janitor nodes with the flag set, waits up to 10s for a remote janitor -/// exposing the gRPC endpoint and logs an error if none is found. -async fn get_compaction_service_client_if_needed( +/// On janitor+compactor nodes, wraps a local `StubCompactionPlannerService`. +/// On compactor-only nodes, connects to a remote janitor via gRPC. +async fn get_compaction_planner_client_if_needed( node_config: &NodeConfig, cluster: &Cluster, -) -> anyhow::Result, anyhow::Error> { - if !node_config.indexer_config.enable_standalone_compactors { +) -> anyhow::Result, anyhow::Error> { + if !node_config.is_service_enabled(QuickwitService::Compactor) { return Ok(None); } - // Only janitor nodes (which host the planner) and indexer nodes (which need - // to know whether to spawn local merge pipelines) care about this service. - if !node_config.is_service_enabled(QuickwitService::Indexer) { - return Ok(None); - } - if node_config.is_service_enabled(QuickwitService::Janitor) - && node_config.is_service_enabled(QuickwitService::Indexer) - { - info!("compaction service enabled on this node"); - return Ok(Some(CompactionServiceClient::new(StubCompactionService))); + if node_config.is_service_enabled(QuickwitService::Janitor) { + info!("compaction planner service enabled on this node"); + return Ok(Some(CompactionPlannerServiceClient::new( + StubCompactionPlannerService, + ))); } let balance_channel = balance_channel_for_service(cluster, QuickwitService::Janitor).await; let found = balance_channel @@ -301,10 +295,10 @@ async fn get_compaction_service_client_if_needed( }) .await; if !found { - bail!("compaction service is enabled but no janitor node was found in the cluster") + bail!("compactor is enabled but no janitor node was found in the cluster") } - info!("remote compaction service detected on janitor node"); - Ok(Some(CompactionServiceClient::from_balance_channel( + info!("remote compaction planner detected on janitor node"); + Ok(Some(CompactionPlannerServiceClient::from_balance_channel( balance_channel, node_config.grpc_config.max_message_size, None, @@ -591,7 +585,7 @@ pub async fn serve_quickwit( // Set up the "control plane proxy" for the metastore. let metastore_through_control_plane = MetastoreServiceClient::new(ControlPlaneMetastore::new( control_plane_client.clone(), - metastore_client, + metastore_client.clone(), )); // Setup ingest service v1. @@ -600,12 +594,12 @@ pub async fn serve_quickwit( .context("failed to start ingest v1 service")?; let compaction_service_client_opt = - get_compaction_service_client_if_needed(&node_config, &cluster) + get_compaction_planner_client_if_needed(&node_config, &cluster) .await .context("failed to initialize compaction service client")?; let indexing_service_opt = if node_config.is_service_enabled(QuickwitService::Indexer) { - let merge_scheduler_mailbox_opt = if compaction_service_client_opt.is_none() { + let merge_scheduler_mailbox_opt = if !node_config.indexer_config.enable_standalone_compactors { Some(spawn_merge_scheduler_service(&universe, &node_config)) } else { None @@ -791,14 +785,20 @@ pub async fn serve_quickwit( let compaction_root_directory = quickwit_common::temp_dir::Builder::default() .tempdir_in(&compaction_dir) .context("failed to create compaction temp directory")?; + // TODO: Real split store let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new( RamStorage::default(), )); + let compaction_client = compaction_service_client_opt + .clone() + .expect("compactor service enabled but no compaction client available"); let compactor_mailbox = start_compactor_service( &universe, + cluster.self_node_id().into(), + compaction_client, &node_config.compactor_config, split_store, - metastore_through_control_plane.clone(), + metastore_client.clone(), storage_resolver.clone(), event_broker.clone(), compaction_root_directory, @@ -1917,18 +1917,22 @@ mod tests { .await .unwrap(); - // Without standalone compactors, no compaction service. + // Without compactor service enabled, no planner client. let mut node_config = NodeConfig::for_test(); node_config.enabled_services = HashSet::from([QuickwitService::Janitor, QuickwitService::Indexer]); - let result = get_compaction_service_client_if_needed(&node_config, &cluster) + let result = get_compaction_planner_client_if_needed(&node_config, &cluster) .await .unwrap(); assert!(result.is_none()); - // With standalone compactors enabled, compaction service client is returned. - node_config.indexer_config.enable_standalone_compactors = true; - let result = get_compaction_service_client_if_needed(&node_config, &cluster) + // With compactor + janitor enabled, planner client is returned (local stub). + node_config.enabled_services = HashSet::from([ + QuickwitService::Janitor, + QuickwitService::Indexer, + QuickwitService::Compactor, + ]); + let result = get_compaction_planner_client_if_needed(&node_config, &cluster) .await .unwrap(); assert!(result.is_some()); @@ -1942,9 +1946,9 @@ mod tests { .unwrap(); let mut node_config = NodeConfig::for_test(); - node_config.enabled_services = HashSet::from([QuickwitService::Indexer]); - node_config.indexer_config.enable_standalone_compactors = true; - let result = get_compaction_service_client_if_needed(&node_config, &cluster).await; + node_config.enabled_services = + HashSet::from([QuickwitService::Indexer, QuickwitService::Compactor]); + let result = get_compaction_planner_client_if_needed(&node_config, &cluster).await; assert!(result.is_err()); } } From 597a81e667608a99c4768a3c9cee183ac5174378 Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Thu, 16 Apr 2026 14:49:00 -0400 Subject: [PATCH 07/11] Merge planner part 1: Metastore (#6305) --- quickwit/Cargo.lock | 2 + quickwit/quickwit-compaction/Cargo.toml | 3 + .../src/planner/compaction_planner.rs | 310 +++++++++++++ .../src/planner/compaction_state.rs | 415 ++++++++++++++++++ .../src/planner/index_config_store.rs | 307 +++++++++++++ .../quickwit-compaction/src/planner/mod.rs | 7 + quickwit/quickwit-serve/src/lib.rs | 11 +- 7 files changed, 1050 insertions(+), 5 deletions(-) create mode 100644 quickwit/quickwit-compaction/src/planner/compaction_planner.rs create mode 100644 quickwit/quickwit-compaction/src/planner/compaction_state.rs create mode 100644 quickwit/quickwit-compaction/src/planner/index_config_store.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 93206bf5d46..73a205987a1 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7158,6 +7158,8 @@ dependencies = [ "quickwit-metastore", "quickwit-proto", "quickwit-storage", + "serde_json", + "time", "tokio", "tracing", ] diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index 88735bb1606..dd817fe6614 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -18,8 +18,11 @@ quickwit-common = { workspace = true } quickwit-config = { workspace = true } quickwit-doc-mapper = { workspace = true } quickwit-indexing = { workspace = true } +quickwit-metastore = { workspace = true } quickwit-proto = { workspace = true } quickwit-storage = { workspace = true } +serde_json = { workspace = true } +time = { workspace = true } tracing = { workspace = true } tokio = { workspace = true } diff --git a/quickwit/quickwit-compaction/src/planner/compaction_planner.rs b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs new file mode 100644 index 00000000000..45d6fe5973c --- /dev/null +++ b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs @@ -0,0 +1,310 @@ +// 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. + +use std::time::Duration; + +use anyhow::Result; +use async_trait::async_trait; +use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitState, +}; +use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService, MetastoreServiceClient}; +use time::OffsetDateTime; +use tracing::error; + +use super::compaction_state::CompactionState; +use super::index_config_store::IndexConfigStore; + +pub struct CompactionPlanner { + state: CompactionState, + index_config_store: IndexConfigStore, + cursor: i64, + metastore: MetastoreServiceClient, +} + +const STARTUP_LOOKBACK: Duration = Duration::from_secs(24 * 60 * 60); + +impl CompactionPlanner { + pub fn new(metastore: MetastoreServiceClient) -> Self { + let cursor = OffsetDateTime::now_utc().unix_timestamp() - STARTUP_LOOKBACK.as_secs() as i64; + CompactionPlanner { + state: CompactionState::new(), + index_config_store: IndexConfigStore::new(metastore.clone()), + cursor, + metastore, + } + } + + async fn ingest_splits(&mut self, splits: Vec) { + for split in splits { + if self.state.is_split_known(&split.split_metadata.split_id) { + continue; + } + let Ok(index_entry) = self + .index_config_store + .get_for_split(&split.split_metadata) + .await + else { + error!(split_id=%split.split_metadata.split_id, "failed to load index config, skipping split"); + continue; + }; + if index_entry.is_split_mature(&split.split_metadata) { + continue; + } + self.cursor = self.cursor.max(split.update_timestamp); + self.state.track_split(split.split_metadata); + } + } + + async fn scan_metastore(&self) -> Result> { + let query = ListSplitsQuery::for_all_indexes() + .with_split_state(SplitState::Published) + .retain_immature(OffsetDateTime::now_utc()) + .with_update_timestamp_gte(self.cursor); + let request = ListSplitsRequest::try_from_list_splits_query(&query)?; + let splits = self + .metastore + .list_splits(request) + .await? + .collect_splits() + .await?; + Ok(splits) + } + + async fn scan_and_plan(&mut self) -> Result<()> { + let splits = self.scan_metastore().await?; + self.ingest_splits(splits).await; + self.run_merge_policies(); + Ok(()) + } + + fn run_merge_policies(&mut self) { + for partition_key in self.state.partition_keys() { + if let Some(index_entry) = self.index_config_store.get(&partition_key.index_uid) { + self.state + .plan_partition(&partition_key, index_entry.merge_policy()); + } + } + } +} + +const SCAN_AND_PLAN_INTERVAL: Duration = Duration::from_secs(5); + +#[derive(Debug)] +struct ScanAndPlan; + +#[async_trait] +impl Actor for CompactionPlanner { + type ObservableState = (); + + fn name(&self) -> String { + "CompactionPlanner".to_string() + } + + fn observable_state(&self) -> Self::ObservableState {} + + async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { + tracing::info!("compaction planner starting, scanning metastore for immature splits"); + if let Err(err) = self.scan_and_plan().await { + error!(error=%err, "error scanning metastore and planning merges"); + } + ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); + Ok(()) + } +} + +#[async_trait] +impl Handler for CompactionPlanner { + type Reply = (); + + async fn handle( + &mut self, + _msg: ScanAndPlan, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + if let Err(err) = self.scan_and_plan().await { + error!(error=%err, "error scanning metastore and planning merges"); + } + self.state.check_heartbeat_timeouts(); + ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use quickwit_common::ServiceStream; + use quickwit_metastore::{ + IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt, Split, SplitMetadata, + SplitState, + }; + use quickwit_proto::metastore::{ + IndexMetadataResponse, ListSplitsResponse, MetastoreError, MockMetastoreService, + }; + use quickwit_proto::types::IndexUid; + + use super::*; + + fn test_split(split_id: &str, index_uid: &IndexUid, update_timestamp: i64) -> Split { + Split { + split_state: SplitState::Published, + update_timestamp, + publish_timestamp: Some(update_timestamp), + split_metadata: SplitMetadata { + split_id: split_id.to_string(), + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + node_id: "test-node".to_string(), + num_docs: 100, + ..Default::default() + }, + } + } + + fn test_index_metadata() -> IndexMetadata { + IndexMetadata::for_test("test-index", "ram:///test-index") + } + + fn test_index_metadata_response(index_metadata: &IndexMetadata) -> IndexMetadataResponse { + IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap() + } + + #[tokio::test] + async fn test_scan_metastore() { + let index_uid = IndexUid::for_test("test-index", 0); + let splits = vec![ + test_split("split-1", &index_uid, 1000), + test_split("split-2", &index_uid, 2000), + ]; + let splits_clone = splits.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_list_splits().returning(move |_| { + let response = ListSplitsResponse::try_from_splits(splits_clone.clone()).unwrap(); + Ok(ServiceStream::from(vec![Ok(response)])) + }); + + let planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + let result = planner.scan_metastore().await.unwrap(); + + assert_eq!(result.len(), 2); + assert_eq!(result[0].split_metadata.split_id, "split-1"); + assert_eq!(result[1].split_metadata.split_id, "split-2"); + } + + #[tokio::test] + async fn test_ingest_splits_dedup_maturity_and_cursor() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + + let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + planner.cursor = 0; + + // Pre-populate: "in-flight" is already being compacted. + planner.state.track_split(SplitMetadata { + split_id: "in-flight".to_string(), + index_uid: index_uid.clone(), + ..Default::default() + }); + + let mut mature_split = test_split("mature", &index_uid, 4000); + mature_split.split_metadata.num_docs = 20_000_000; + + let splits = vec![ + test_split("in-flight", &index_uid, 1000), + test_split("fresh", &index_uid, 3000), + mature_split, + ]; + + planner.ingest_splits(splits).await; + + assert!(planner.state.is_split_known("fresh")); + assert!(planner.state.is_split_known("in-flight")); + assert!(!planner.state.is_split_known("mature")); + assert_eq!(planner.cursor, 3000); + } + + #[tokio::test] + async fn test_scan_and_plan_metastore_error() { + let mut mock = MockMetastoreService::new(); + mock.expect_list_splits().returning(|_| { + Err(MetastoreError::Internal { + message: "test error".to_string(), + cause: String::new(), + }) + }); + + let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + let original_cursor = planner.cursor; + + let result = planner.scan_and_plan().await; + assert!(result.is_err()); + assert_eq!(planner.cursor, original_cursor); + } + + #[tokio::test] + async fn test_ingest_splits_skips_on_config_error() { + let index_uid = IndexUid::for_test("missing-index", 0); + let splits = vec![test_split("orphan", &index_uid, 1000)]; + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata().returning(|_| { + Err(MetastoreError::Internal { + message: "test error".to_string(), + cause: String::new(), + }) + }); + + let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + planner.cursor = 0; + planner.ingest_splits(splits).await; + + assert!(!planner.state.is_split_known("orphan")); + } + + #[tokio::test] + async fn test_scan_and_plan_happy_path() { + let index_metadata = test_index_metadata(); + let index_metadata_response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let splits = vec![ + test_split("s1", &index_uid, 5000), + test_split("s2", &index_uid, 6000), + ]; + let splits_clone = splits.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_list_splits().returning(move |_| { + let response = ListSplitsResponse::try_from_splits(splits_clone.clone()).unwrap(); + Ok(ServiceStream::from(vec![Ok(response)])) + }); + mock.expect_index_metadata() + .returning(move |_| Ok(index_metadata_response.clone())); + + let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + planner.cursor = 0; + planner.scan_and_plan().await.unwrap(); + + assert!(planner.state.is_split_known("s1")); + assert!(planner.state.is_split_known("s2")); + assert_eq!(planner.cursor, 6000); + } +} diff --git a/quickwit/quickwit-compaction/src/planner/compaction_state.rs b/quickwit/quickwit-compaction/src/planner/compaction_state.rs new file mode 100644 index 00000000000..1d1ea07e91d --- /dev/null +++ b/quickwit/quickwit-compaction/src/planner/compaction_state.rs @@ -0,0 +1,415 @@ +// 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. + +use std::collections::{HashMap, HashSet, VecDeque}; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use quickwit_indexing::merge_policy::{MergeOperation, MergePolicy}; +use quickwit_metastore::SplitMetadata; +use quickwit_proto::compaction::{CompactionFailure, CompactionInProgress, CompactionSuccess}; +use quickwit_proto::types::{DocMappingUid, IndexUid, NodeId, SourceId, SplitId}; +use tracing::{error, info, warn}; + +const HEARTBEAT_TIMEOUT: Duration = Duration::from_secs(60); + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct CompactionPartitionKey { + pub index_uid: IndexUid, + pub source_id: SourceId, + pub partition_id: u64, + pub doc_mapping_uid: DocMappingUid, +} + +impl CompactionPartitionKey { + pub fn from_split(split: &SplitMetadata) -> Self { + CompactionPartitionKey { + index_uid: split.index_uid.clone(), + source_id: split.source_id.clone(), + partition_id: split.partition_id, + doc_mapping_uid: split.doc_mapping_uid, + } + } +} + +struct InFlightCompaction { + task_id: String, + split_ids: Vec, + node_id: NodeId, + last_heartbeat: Instant, +} + +/// Tracks all split-level state for the compaction planner: +/// which splits need compaction, which are in-flight, and which +/// operations are pending assignment to workers. +pub struct CompactionState { + needs_compaction: HashMap>, + needs_compaction_split_ids: HashSet, + in_flight: HashMap, + in_flight_split_ids: HashSet, + /// TODO: add index_uid and source_id to MergeOperation so we don't need the partition key + /// here. + pending_operations: VecDeque<(CompactionPartitionKey, MergeOperation)>, +} + +impl CompactionState { + pub fn new() -> Self { + CompactionState { + needs_compaction: HashMap::new(), + needs_compaction_split_ids: HashSet::new(), + in_flight: HashMap::new(), + in_flight_split_ids: HashSet::new(), + pending_operations: VecDeque::new(), + } + } + + /// Returns true if the split is already tracked (either awaiting compaction or in-flight). + pub fn is_split_known(&self, split_id: &str) -> bool { + self.needs_compaction_split_ids.contains(split_id) + || self.in_flight_split_ids.contains(split_id) + } + + /// Adds a split to the needs_compaction set. + pub fn track_split(&mut self, split: SplitMetadata) { + let split_id = split.split_id().to_string(); + let key = CompactionPartitionKey::from_split(&split); + self.needs_compaction_split_ids.insert(split_id); + self.needs_compaction.entry(key).or_default().push(split); + } + + /// Returns the partition keys for iteration. The caller drives the loop. + pub fn partition_keys(&self) -> Vec { + self.needs_compaction.keys().cloned().collect() + } + + /// Runs a merge policy on a single partition and queues the resulting operations. + pub fn plan_partition( + &mut self, + partition_key: &CompactionPartitionKey, + merge_policy: &Arc, + ) { + let Some(splits) = self.needs_compaction.get_mut(partition_key) else { + return; + }; + for operation in merge_policy.operations(splits) { + for split in operation.splits_as_slice() { + self.needs_compaction_split_ids.remove(split.split_id()); + self.in_flight_split_ids + .insert(split.split_id().to_string()); + } + self.pending_operations + .push_back((partition_key.clone(), operation)); + } + if splits.is_empty() { + self.needs_compaction.remove(partition_key); + } + } + + pub fn process_successes(&mut self, successes: &[CompactionSuccess]) { + for success in successes { + if let Some(inflight) = self.in_flight.remove(&success.task_id) { + info!(task_id=%success.task_id, "compaction task completed"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + pub fn process_failures(&mut self, failures: &[CompactionFailure]) { + for failure in failures { + if let Some(inflight) = self.in_flight.remove(&failure.task_id) { + warn!(task_id=%failure.task_id, error=%failure.error_message, "compaction task failed"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + pub fn update_heartbeats(&mut self, node_id: &NodeId, in_progress: &[CompactionInProgress]) { + for task in in_progress { + if let Some(inflight) = self.in_flight.get_mut(&task.task_id) { + inflight.last_heartbeat = Instant::now(); + } else { + // Task not tracked — start tracking it. Happens when + // workers report tasks the planner doesn't know about yet + // (e.g. after planner start). + for split_id in &task.split_ids { + self.in_flight_split_ids.insert(split_id.clone()); + self.needs_compaction_split_ids.remove(split_id.as_str()); + } + self.in_flight.insert( + task.task_id.clone(), + InFlightCompaction { + task_id: task.task_id.clone(), + split_ids: task.split_ids.clone(), + node_id: node_id.clone(), + last_heartbeat: Instant::now(), + }, + ); + } + } + } + + pub fn check_heartbeat_timeouts(&mut self) { + let now = Instant::now(); + let timed_out_task_ids: Vec = self + .in_flight + .iter() + .filter(|(_, inflight)| now.duration_since(inflight.last_heartbeat) > HEARTBEAT_TIMEOUT) + .map(|(task_id, _)| task_id.clone()) + .collect(); + + for task_id in timed_out_task_ids { + if let Some(inflight) = self.in_flight.remove(&task_id) { + error!(task_id=%task_id, node_id=%inflight.node_id, "compaction task timed out"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + /// Pops up to `count` pending operations for assignment. + pub fn pop_pending(&mut self, count: usize) -> Vec<(CompactionPartitionKey, MergeOperation)> { + let count = count.min(self.pending_operations.len()); + let mut operations = Vec::with_capacity(count); + for _ in 0..count { + operations.push(self.pending_operations.pop_front().unwrap()); + } + operations + } + + /// Records that an operation has been assigned to a worker. + pub fn record_assignment(&mut self, task_id: String, split_ids: Vec, node_id: NodeId) { + self.in_flight.insert( + task_id.clone(), + InFlightCompaction { + task_id, + split_ids, + node_id, + last_heartbeat: Instant::now(), + }, + ); + } +} + +#[cfg(test)] +mod tests { + use std::time::Duration; + + use quickwit_config::IndexingSettings; + use quickwit_config::merge_policy_config::{ + ConstWriteAmplificationMergePolicyConfig, MergePolicyConfig, + }; + use quickwit_indexing::merge_policy::merge_policy_from_settings; + use quickwit_proto::types::IndexUid; + + use super::*; + + fn test_merge_policy() -> Arc { + let settings = IndexingSettings { + merge_policy: MergePolicyConfig::ConstWriteAmplification( + ConstWriteAmplificationMergePolicyConfig { + merge_factor: 2, + max_merge_factor: 2, + ..Default::default() + }, + ), + ..Default::default() + }; + merge_policy_from_settings(&settings) + } + + fn test_split(split_id: &str, index_uid: &IndexUid) -> SplitMetadata { + SplitMetadata { + split_id: split_id.to_string(), + index_uid: index_uid.clone(), + source_id: "test-source".to_string(), + node_id: "test-node".to_string(), + num_docs: 1000, + create_timestamp: time::OffsetDateTime::now_utc().unix_timestamp(), + maturity: quickwit_metastore::SplitMaturity::Immature { + maturation_period: Duration::from_secs(3600), + }, + ..Default::default() + } + } + + #[test] + fn test_track_and_is_known() { + let index_uid = IndexUid::for_test("test-index", 0); + let mut state = CompactionState::new(); + + assert!(!state.is_split_known("s1")); + + state.track_split(test_split("s1", &index_uid)); + assert!(state.is_split_known("s1")); + assert!(!state.is_split_known("s2")); + } + + #[test] + fn test_track_split_partitions_correctly() { + let index_uid = IndexUid::for_test("test-index", 0); + let mut state = CompactionState::new(); + + state.track_split(test_split("s1", &index_uid)); + state.track_split(test_split("s2", &index_uid)); + + assert_eq!(state.partition_keys().len(), 1); + } + + #[test] + fn test_plan_partition_moves_splits_to_in_flight() { + let index_uid = IndexUid::for_test("test-index", 0); + let merge_policy = test_merge_policy(); + let mut state = CompactionState::new(); + + state.track_split(test_split("s0", &index_uid)); + state.track_split(test_split("s1", &index_uid)); + + let keys = state.partition_keys(); + assert_eq!(keys.len(), 1); + + state.plan_partition(&keys[0], &merge_policy); + + // Splits moved from needs_compaction to in_flight. + assert!(!state.pending_operations.is_empty()); + for (_, op) in &state.pending_operations { + for split in op.splits_as_slice() { + assert!(!state.needs_compaction_split_ids.contains(split.split_id())); + assert!(state.in_flight_split_ids.contains(split.split_id())); + } + } + } + + #[test] + fn test_process_successes_and_failures_clear_in_flight() { + let node_id = NodeId::from("worker-1"); + let mut state = CompactionState::new(); + + // Simulate what plan_partition + record_assignment does: + // split IDs go into in_flight_split_ids, then into InFlightCompaction. + state.in_flight_split_ids.insert("s1".to_string()); + state.in_flight_split_ids.insert("s2".to_string()); + state.record_assignment( + "task-1".to_string(), + vec!["s1".to_string(), "s2".to_string()], + node_id.clone(), + ); + + state.in_flight_split_ids.insert("s3".to_string()); + state.record_assignment("task-2".to_string(), vec!["s3".to_string()], node_id); + + assert!(state.is_split_known("s1")); + assert!(state.is_split_known("s3")); + + state.process_successes(&[CompactionSuccess { + task_id: "task-1".to_string(), + merged_split_id: "merged-1".to_string(), + }]); + assert!(!state.is_split_known("s1")); + assert!(!state.is_split_known("s2")); + + state.process_failures(&[CompactionFailure { + task_id: "task-2".to_string(), + error_message: "boom".to_string(), + }]); + assert!(!state.is_split_known("s3")); + } + + #[test] + fn test_update_heartbeats_adopts_unknown_tasks() { + let node_id = NodeId::from("worker-1"); + let mut state = CompactionState::new(); + + // Simulate a split that was tracked as needing compaction. + let index_uid = IndexUid::for_test("test-index", 0); + state.track_split(test_split("s1", &index_uid)); + assert!(state.needs_compaction_split_ids.contains("s1")); + + // Worker reports an in-progress task the planner doesn't know about. + state.update_heartbeats( + &node_id, + &[CompactionInProgress { + task_id: "task-1".to_string(), + index_uid: Some(index_uid), + source_id: "test-source".to_string(), + split_ids: vec!["s1".to_string()], + }], + ); + + // Split moved from needs_compaction to in_flight. + assert!(state.in_flight_split_ids.contains("s1")); + assert!(!state.needs_compaction_split_ids.contains("s1")); + } + + #[test] + fn test_pop_pending_and_record_assignment() { + let index_uid = IndexUid::for_test("test-index", 0); + let merge_policy = test_merge_policy(); + let mut state = CompactionState::new(); + + state.track_split(test_split("s0", &index_uid)); + state.track_split(test_split("s1", &index_uid)); + + let keys = state.partition_keys(); + state.plan_partition(&keys[0], &merge_policy); + + let pending = state.pop_pending(1); + assert_eq!(pending.len(), 1); + + let (_, operation) = &pending[0]; + let split_ids: Vec = operation + .splits_as_slice() + .iter() + .map(|s| s.split_id().to_string()) + .collect(); + + // Splits are already in in_flight_split_ids from plan_partition. + state.record_assignment("task-1".to_string(), split_ids.clone(), NodeId::from("w1")); + + for split_id in &split_ids { + assert!(state.is_split_known(split_id)); + } + } + + #[test] + fn test_pop_pending_respects_count() { + let index_uid = IndexUid::for_test("test-index", 0); + let merge_policy = test_merge_policy(); + let mut state = CompactionState::new(); + + // With merge_factor=2, 4 splits produces 2 operations. + for i in 0..4 { + state.track_split(test_split(&format!("s{i}"), &index_uid)); + } + + let keys = state.partition_keys(); + state.plan_partition(&keys[0], &merge_policy); + + let total_pending = state.pending_operations.len(); + assert!(total_pending >= 2); + + let popped = state.pop_pending(1); + assert_eq!(popped.len(), 1); + assert_eq!(state.pending_operations.len(), total_pending - 1); + + // Asking for more than available returns what's there. + let rest = state.pop_pending(100); + assert_eq!(rest.len(), total_pending - 1); + assert!(state.pending_operations.is_empty()); + } +} diff --git a/quickwit/quickwit-compaction/src/planner/index_config_store.rs b/quickwit/quickwit-compaction/src/planner/index_config_store.rs new file mode 100644 index 00000000000..e47044d21e3 --- /dev/null +++ b/quickwit/quickwit-compaction/src/planner/index_config_store.rs @@ -0,0 +1,307 @@ +// 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. + +use std::collections::HashMap; +use std::sync::Arc; + +use quickwit_config::{IndexConfig, build_doc_mapper}; +use quickwit_doc_mapper::DocMapper; +use quickwit_indexing::merge_policy::{MergePolicy, merge_policy_from_settings}; +use quickwit_metastore::{IndexMetadataResponseExt, SplitMaturity, SplitMetadata}; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; +use quickwit_proto::types::{DocMappingUid, IndexUid}; + +/// Everything the planner needs to know about a single index. +#[derive(Clone)] +pub struct IndexEntry { + config: IndexConfig, + merge_policy: Arc, + doc_mappers: HashMap>, +} + +impl IndexEntry { + pub fn is_split_mature(&self, split: &SplitMetadata) -> bool { + matches!( + self.merge_policy + .split_maturity(split.num_docs, split.num_merge_ops), + SplitMaturity::Mature + ) + } + + pub fn merge_policy(&self) -> &Arc { + &self.merge_policy + } + + pub fn doc_mapping_json(&self) -> String { + serde_json::to_string(&self.config.doc_mapping) + .expect("doc mapping serialization should not fail") + } + + pub fn search_settings_json(&self) -> String { + serde_json::to_string(&self.config.search_settings) + .expect("search settings serialization should not fail") + } + + pub fn indexing_settings_json(&self) -> String { + serde_json::to_string(&self.config.indexing_settings) + .expect("indexing settings serialization should not fail") + } + + pub fn retention_policy_json(&self) -> String { + match &self.config.retention_policy_opt { + Some(policy) => serde_json::to_string(policy) + .expect("retention policy serialization should not fail"), + None => String::new(), + } + } + + pub fn index_storage_uri(&self) -> String { + self.config.index_uri.to_string() + } +} + +/// Caches per-index configuration, merge policies, and doc mappers. +/// Fetches from the metastore on demand. All accessors panic if called +/// for an index that hasn't been loaded. +pub struct IndexConfigStore { + indexes: HashMap, + metastore: MetastoreServiceClient, +} + +impl IndexConfigStore { + pub fn new(metastore: MetastoreServiceClient) -> Self { + IndexConfigStore { + indexes: HashMap::new(), + metastore, + } + } + + /// Fetches an index config from the metastore and builds the derived + /// artifacts. + async fn fetch_index_config( + &mut self, + index_uid: &IndexUid, + doc_mapping_uid: &DocMappingUid, + ) -> anyhow::Result<()> { + let response = self + .metastore + .index_metadata(IndexMetadataRequest { + index_uid: Some(index_uid.clone()), + index_id: None, + }) + .await?; + let index_metadata = response.deserialize_index_metadata()?; + + let doc_mapper = build_doc_mapper( + &index_metadata.index_config.doc_mapping, + &index_metadata.index_config.search_settings, + )?; + let merge_policy = + merge_policy_from_settings(&index_metadata.index_config.indexing_settings); + + let mut doc_mappers = HashMap::new(); + doc_mappers.insert(*doc_mapping_uid, doc_mapper); + let entry = IndexEntry { + config: index_metadata.index_config, + merge_policy, + doc_mappers, + }; + self.indexes.insert(index_uid.clone(), entry); + Ok(()) + } + + /// Gets the index entry, fetching from the metastore if not cached. + pub async fn get_or_fetch( + &mut self, + index_uid: &IndexUid, + doc_mapping_uid: &DocMappingUid, + ) -> anyhow::Result<&IndexEntry> { + match self.indexes.get(index_uid) { + Some(entry) if entry.doc_mappers.contains_key(doc_mapping_uid) => {} + _ => self.fetch_index_config(index_uid, doc_mapping_uid).await?, + } + Ok(&self.indexes[index_uid]) + } + + pub async fn get_for_split(&mut self, split: &SplitMetadata) -> anyhow::Result<&IndexEntry> { + self.get_or_fetch(&split.index_uid, &split.doc_mapping_uid) + .await + } + + pub fn get(&self, index_uid: &IndexUid) -> Option<&IndexEntry> { + self.indexes.get(index_uid) + } +} + +#[cfg(test)] +mod tests { + use quickwit_metastore::{IndexMetadata, IndexMetadataResponseExt}; + use quickwit_proto::metastore::{IndexMetadataResponse, MetastoreError, MockMetastoreService}; + + use super::*; + + fn test_index_metadata() -> IndexMetadata { + IndexMetadata::for_test("test-index", "ram:///test-index") + } + + fn test_index_metadata_response(index_metadata: &IndexMetadata) -> IndexMetadataResponse { + IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap() + } + + #[tokio::test] + async fn test_get_or_fetch_loads_and_caches() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + let doc_mapping_uid = DocMappingUid::default(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .times(1) + .returning(move |_| Ok(response.clone())); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + + // First call fetches from metastore. + let entry = store + .get_or_fetch(&index_uid, &doc_mapping_uid) + .await + .unwrap(); + assert!(!entry.doc_mapping_json().is_empty()); + assert!(!entry.search_settings_json().is_empty()); + assert!(!entry.indexing_settings_json().is_empty()); + assert!(!entry.index_storage_uri().is_empty()); + + // Second call hits cache (times(1) would panic otherwise). + store + .get_or_fetch(&index_uid, &doc_mapping_uid) + .await + .unwrap(); + } + + #[tokio::test] + async fn test_get_or_fetch_metastore_error() { + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata().returning(|_| { + Err(MetastoreError::Internal { + message: "test error".to_string(), + cause: String::new(), + }) + }); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + let result = store + .get_or_fetch(&IndexUid::for_test("missing", 0), &DocMappingUid::default()) + .await; + assert!(result.is_err()); + } + + #[tokio::test] + async fn test_get_returns_none_before_fetch() { + let mock = MockMetastoreService::new(); + let store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + assert!(store.get(&IndexUid::for_test("test-index", 0)).is_none()); + } + + #[tokio::test] + async fn test_get_returns_some_after_fetch() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + store + .get_or_fetch(&index_uid, &DocMappingUid::default()) + .await + .unwrap(); + + assert!(store.get(&index_uid).is_some()); + } + + #[tokio::test] + async fn test_get_for_split() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + let split = SplitMetadata { + split_id: "split-1".to_string(), + index_uid: index_uid.clone(), + ..Default::default() + }; + + let entry = store.get_for_split(&split).await.unwrap(); + assert!(!entry.index_storage_uri().is_empty()); + } + + #[tokio::test] + async fn test_is_split_mature() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + let entry = store + .get_or_fetch(&index_uid, &DocMappingUid::default()) + .await + .unwrap(); + + let small_split = SplitMetadata { + num_docs: 100, + num_merge_ops: 0, + ..Default::default() + }; + assert!(!entry.is_split_mature(&small_split)); + + let large_split = SplitMetadata { + num_docs: 20_000_000, + num_merge_ops: 0, + ..Default::default() + }; + assert!(entry.is_split_mature(&large_split)); + } + + #[tokio::test] + async fn test_retention_policy_json_empty_when_none() { + let index_metadata = test_index_metadata(); + let response = test_index_metadata_response(&index_metadata); + let index_uid = index_metadata.index_uid.clone(); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + + let mut store = IndexConfigStore::new(MetastoreServiceClient::from_mock(mock)); + let entry = store + .get_or_fetch(&index_uid, &DocMappingUid::default()) + .await + .unwrap(); + + // Default test index has no retention policy. + assert!(entry.retention_policy_json().is_empty()); + } +} diff --git a/quickwit/quickwit-compaction/src/planner/mod.rs b/quickwit/quickwit-compaction/src/planner/mod.rs index 5ccd750a20c..e780b30e0ef 100644 --- a/quickwit/quickwit-compaction/src/planner/mod.rs +++ b/quickwit/quickwit-compaction/src/planner/mod.rs @@ -12,6 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#[allow(dead_code)] +mod compaction_planner; mod compaction_service; +#[allow(dead_code)] +mod compaction_state; +#[allow(dead_code)] +mod index_config_store; +pub use compaction_planner::CompactionPlanner; pub use compaction_service::StubCompactionPlannerService; diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 3eda56d5e5e..9dafbfe9d4a 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -599,11 +599,12 @@ pub async fn serve_quickwit( .context("failed to initialize compaction service client")?; let indexing_service_opt = if node_config.is_service_enabled(QuickwitService::Indexer) { - let merge_scheduler_mailbox_opt = if !node_config.indexer_config.enable_standalone_compactors { - Some(spawn_merge_scheduler_service(&universe, &node_config)) - } else { - None - }; + let merge_scheduler_mailbox_opt = + if !node_config.indexer_config.enable_standalone_compactors { + Some(spawn_merge_scheduler_service(&universe, &node_config)) + } else { + None + }; let indexing_service = start_indexing_service( &universe, &node_config, From 7fcb4cefdd6283efe4a9b91559859ff9ee374dcf Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Thu, 16 Apr 2026 14:51:00 -0400 Subject: [PATCH 08/11] Merge planner part 2: gRPCs (#6310) --- quickwit/Cargo.lock | 1 + quickwit/quickwit-compaction/Cargo.toml | 1 + .../src/compactor_supervisor.rs | 8 +- .../src/planner/compaction_planner.rs | 290 +++++++++++++++--- .../src/planner/compaction_service.rs | 36 --- .../quickwit-compaction/src/planner/mod.rs | 3 - .../protos/quickwit/compaction.proto | 20 +- .../codegen/quickwit/quickwit.compaction.rs | 270 ++-------------- quickwit/quickwit-proto/src/compaction/mod.rs | 15 + quickwit/quickwit-serve/src/lib.rs | 79 +++-- 10 files changed, 374 insertions(+), 349 deletions(-) delete mode 100644 quickwit/quickwit-compaction/src/planner/compaction_service.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 73a205987a1..68d9ad60f96 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7162,6 +7162,7 @@ dependencies = [ "time", "tokio", "tracing", + "ulid", ] [[package]] diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index dd817fe6614..c018327c8c4 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -25,6 +25,7 @@ serde_json = { workspace = true } time = { workspace = true } tracing = { workspace = true } tokio = { workspace = true } +ulid = { workspace = true } [dev-dependencies] quickwit-actors = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index e2809c4e8d3..e1b27eb4a30 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -189,20 +189,22 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::temp_dir::TempDirectory; - use quickwit_proto::compaction::CompactionPlannerServiceClient; + use quickwit_proto::compaction::{ + CompactionPlannerServiceClient, MockCompactionPlannerService, + }; use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; use quickwit_proto::types::NodeId; use quickwit_storage::{RamStorage, StorageResolver}; use super::*; use crate::compaction_pipeline::tests::test_pipeline; - use crate::planner::StubCompactionPlannerService; fn test_supervisor(num_slots: usize) -> CompactorSupervisor { let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); - let compaction_client = CompactionPlannerServiceClient::new(StubCompactionPlannerService); + let compaction_client = + CompactionPlannerServiceClient::from_mock(MockCompactionPlannerService::new()); CompactorSupervisor::new( NodeId::from("test-node"), compaction_client, diff --git a/quickwit/quickwit-compaction/src/planner/compaction_planner.rs b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs index 45d6fe5973c..0fb6ccc6d50 100644 --- a/quickwit/quickwit-compaction/src/planner/compaction_planner.rs +++ b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs @@ -12,20 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::fmt::Debug; use std::time::Duration; use anyhow::Result; use async_trait::async_trait; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; +use quickwit_indexing::merge_policy::MergeOperation; use quickwit_metastore::{ ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitState, }; +use quickwit_proto::compaction::{ + CompactionResult, MergeTaskAssignment, ReportStatusRequest, ReportStatusResponse, +}; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService, MetastoreServiceClient}; +use quickwit_proto::types::{IndexUid, NodeId, SourceId}; use time::OffsetDateTime; use tracing::error; +use ulid::Ulid; use super::compaction_state::CompactionState; -use super::index_config_store::IndexConfigStore; +use super::index_config_store::{IndexConfigStore, IndexEntry}; pub struct CompactionPlanner { state: CompactionState, @@ -34,6 +41,75 @@ pub struct CompactionPlanner { metastore: MetastoreServiceClient, } +impl Debug for CompactionPlanner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("CompactionPlanner") + .field("cursor", &self.cursor) + .finish() + } +} + +const SCAN_AND_PLAN_INTERVAL: Duration = Duration::from_secs(5); +/// On initialization, we want to wait for two intervals to allow any in-progress workers to report +/// their progress, preventing us from frivolously rescheduling work. +const INITIAL_SCAN_AND_PLAN_INTERVAL: Duration = SCAN_AND_PLAN_INTERVAL.saturating_mul(2); + +#[derive(Debug)] +struct ScanAndPlan; + +#[async_trait] +impl Actor for CompactionPlanner { + type ObservableState = (); + + fn name(&self) -> String { + "CompactionPlanner".to_string() + } + + fn observable_state(&self) -> Self::ObservableState {} + + async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { + tracing::info!("compaction planner starting, scanning metastore for immature splits"); + ctx.schedule_self_msg(INITIAL_SCAN_AND_PLAN_INTERVAL, ScanAndPlan); + Ok(()) + } +} + +#[async_trait] +impl Handler for CompactionPlanner { + type Reply = (); + + async fn handle( + &mut self, + _msg: ScanAndPlan, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + if let Err(err) = self.scan_and_plan().await { + error!(error=%err, "error scanning metastore and planning merges"); + } + self.state.check_heartbeat_timeouts(); + ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); + Ok(()) + } +} + +#[async_trait] +impl Handler for CompactionPlanner { + type Reply = CompactionResult; + + async fn handle( + &mut self, + msg: ReportStatusRequest, + _ctx: &ActorContext, + ) -> Result, ActorExitStatus> { + let node_id = NodeId::from(msg.node_id); + self.state.process_successes(&msg.successes); + self.state.process_failures(&msg.failures); + self.state.update_heartbeats(&node_id, &msg.in_progress); + let new_tasks = self.assign_tasks(&node_id, msg.available_slots); + Ok(Ok(ReportStatusResponse { new_tasks })) + } +} + const STARTUP_LOOKBACK: Duration = Duration::from_secs(24 * 60 * 60); impl CompactionPlanner { @@ -98,58 +174,79 @@ impl CompactionPlanner { } } } -} - -const SCAN_AND_PLAN_INTERVAL: Duration = Duration::from_secs(5); - -#[derive(Debug)] -struct ScanAndPlan; - -#[async_trait] -impl Actor for CompactionPlanner { - type ObservableState = (); - - fn name(&self) -> String { - "CompactionPlanner".to_string() - } - fn observable_state(&self) -> Self::ObservableState {} + fn assign_tasks(&mut self, node_id: &NodeId, available_slots: u32) -> Vec { + let pending = self.state.pop_pending(available_slots as usize); + let mut assignments = Vec::with_capacity(pending.len()); - async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { - tracing::info!("compaction planner starting, scanning metastore for immature splits"); - if let Err(err) = self.scan_and_plan().await { - error!(error=%err, "error scanning metastore and planning merges"); + for (partition_key, operation) in pending { + let task_id = Ulid::new().to_string(); + let Some(index_entry) = self.index_config_store.get(&partition_key.index_uid) else { + error!(index_uid=%partition_key.index_uid, "index config not found for pending operation, skipping"); + continue; + }; + let assignment = build_task_assignment( + &task_id, + index_entry, + &operation, + &partition_key.index_uid, + &partition_key.source_id, + ); + + let split_ids = operation + .splits_as_slice() + .iter() + .map(|s| s.split_id().to_string()) + .collect(); + self.state + .record_assignment(task_id, split_ids, node_id.clone()); + + assignments.push(assignment); } - ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); - Ok(()) + assignments } } -#[async_trait] -impl Handler for CompactionPlanner { - type Reply = (); - - async fn handle( - &mut self, - _msg: ScanAndPlan, - ctx: &ActorContext, - ) -> Result<(), ActorExitStatus> { - if let Err(err) = self.scan_and_plan().await { - error!(error=%err, "error scanning metastore and planning merges"); - } - self.state.check_heartbeat_timeouts(); - ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); - Ok(()) +fn build_task_assignment( + task_id: &str, + index_entry: &IndexEntry, + operation: &MergeOperation, + index_uid: &IndexUid, + source_id: &SourceId, +) -> MergeTaskAssignment { + MergeTaskAssignment { + task_id: task_id.to_string(), + splits_metadata_json: operation + .splits_as_slice() + .iter() + .map(|s| { + serde_json::to_string(s).expect("split metadata serialization should not fail") + }) + .collect(), + doc_mapping_json: index_entry.doc_mapping_json(), + search_settings_json: index_entry.search_settings_json(), + indexing_settings_json: index_entry.indexing_settings_json(), + retention_policy_json: index_entry.retention_policy_json(), + index_uid: Some(index_uid.clone()), + source_id: source_id.to_string(), + index_storage_uri: index_entry.index_storage_uri(), } } #[cfg(test)] mod tests { + use std::time::Duration; + use quickwit_common::ServiceStream; + use quickwit_config::IndexingSettings; + use quickwit_config::merge_policy_config::{ + ConstWriteAmplificationMergePolicyConfig, MergePolicyConfig, + }; use quickwit_metastore::{ - IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt, Split, SplitMetadata, - SplitState, + IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt, Split, SplitMaturity, + SplitMetadata, SplitState, }; + use quickwit_proto::compaction::CompactionSuccess; use quickwit_proto::metastore::{ IndexMetadataResponse, ListSplitsResponse, MetastoreError, MockMetastoreService, }; @@ -168,6 +265,10 @@ mod tests { source_id: "test-source".to_string(), node_id: "test-node".to_string(), num_docs: 100, + create_timestamp: OffsetDateTime::now_utc().unix_timestamp(), + maturity: SplitMaturity::Immature { + maturation_period: Duration::from_secs(3600), + }, ..Default::default() }, } @@ -177,6 +278,22 @@ mod tests { IndexMetadata::for_test("test-index", "ram:///test-index") } + /// Returns an IndexMetadata with merge_factor=2 so two splits trigger a merge. + fn test_index_metadata_with_merge_factor_2() -> IndexMetadata { + let mut metadata = test_index_metadata(); + metadata.index_config.indexing_settings = IndexingSettings { + merge_policy: MergePolicyConfig::ConstWriteAmplification( + ConstWriteAmplificationMergePolicyConfig { + merge_factor: 2, + max_merge_factor: 2, + ..Default::default() + }, + ), + ..Default::default() + }; + metadata + } + fn test_index_metadata_response(index_metadata: &IndexMetadata) -> IndexMetadataResponse { IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap() } @@ -307,4 +424,99 @@ mod tests { assert!(planner.state.is_split_known("s2")); assert_eq!(planner.cursor, 6000); } + + /// Helper: creates a planner with merge_factor=2, ingests the given splits, + /// and runs merge policies. Returns the planner ready for `assign_tasks`. + async fn planner_with_pending_merges(split_ids: &[&str]) -> (CompactionPlanner, IndexUid) { + let index_metadata = test_index_metadata_with_merge_factor_2(); + let index_uid = index_metadata.index_uid.clone(); + let response = test_index_metadata_response(&index_metadata); + + let mut mock = MockMetastoreService::new(); + mock.expect_index_metadata() + .returning(move |_| Ok(response.clone())); + mock.expect_list_splits().returning(|_| { + Ok(ServiceStream::from(vec![Ok( + ListSplitsResponse::try_from_splits(Vec::new()).unwrap(), + )])) + }); + + let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + planner.cursor = 0; + + let splits: Vec = split_ids + .iter() + .enumerate() + .map(|(i, id)| test_split(id, &index_uid, (i + 1) as i64 * 1000)) + .collect(); + planner.ingest_splits(splits).await; + planner.run_merge_policies(); + (planner, index_uid) + } + + #[tokio::test] + async fn test_assign_tasks_returns_assignments_and_drains_queue() { + let (mut planner, index_uid) = planner_with_pending_merges(&["s1", "s2"]).await; + let node_id = NodeId::from("worker-1"); + + // First call: get the assignment. + let assignments = planner.assign_tasks(&node_id, 10); + assert_eq!(assignments.len(), 1); + + let assignment = &assignments[0]; + assert!(!assignment.task_id.is_empty()); + assert_eq!(assignment.splits_metadata_json.len(), 2); + assert_eq!(assignment.index_uid, Some(index_uid)); + assert_eq!(assignment.source_id, "test-source"); + assert!(!assignment.doc_mapping_json.is_empty()); + assert!(!assignment.index_storage_uri.is_empty()); + + // Second call: queue is drained, no more assignments. + let assignments = planner.assign_tasks(&node_id, 10); + assert!(assignments.is_empty()); + } + + #[tokio::test] + async fn test_assign_tasks_respects_available_slots() { + // 4 splits with merge_factor=2 produces 2 merge operations. + let (mut planner, _) = planner_with_pending_merges(&["s1", "s2", "s3", "s4"]).await; + let node_id = NodeId::from("worker-1"); + + // Request only 1 slot. + let assignments = planner.assign_tasks(&node_id, 1); + assert_eq!(assignments.len(), 1); + + // The remaining operation is still pending. + let assignments = planner.assign_tasks(&node_id, 10); + assert_eq!(assignments.len(), 1); + } + + #[tokio::test] + async fn test_report_status_success_frees_splits_for_future_merges() { + let (mut planner, index_uid) = planner_with_pending_merges(&["s1", "s2"]).await; + let node_id = NodeId::from("worker-1"); + + let assignments = planner.assign_tasks(&node_id, 10); + assert_eq!(assignments.len(), 1); + let task_id = assignments[0].task_id.clone(); + + // Report success for the task. + planner.state.process_successes(&[CompactionSuccess { + task_id, + merged_split_id: "merged-1".to_string(), + }]); + + // The original splits are no longer tracked. Re-ingesting them + // (simulating the merged output being immature) creates new work. + let new_splits = vec![ + test_split("s5", &index_uid, 5000), + test_split("s6", &index_uid, 6000), + ]; + planner.ingest_splits(new_splits).await; + planner.run_merge_policies(); + + let assignments = planner.assign_tasks(&node_id, 10); + assert_eq!(assignments.len(), 1); + assert_eq!(assignments[0].splits_metadata_json.len(), 2); + } } diff --git a/quickwit/quickwit-compaction/src/planner/compaction_service.rs b/quickwit/quickwit-compaction/src/planner/compaction_service.rs deleted file mode 100644 index 340ff4e0d5c..00000000000 --- a/quickwit/quickwit-compaction/src/planner/compaction_service.rs +++ /dev/null @@ -1,36 +0,0 @@ -// 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. - -use async_trait::async_trait; -use quickwit_proto::compaction::{ - CompactionPlannerService, CompactionResult, PingRequest, PingResponse, ReportStatusRequest, - ReportStatusResponse, -}; - -#[derive(Debug, Clone)] -pub struct StubCompactionPlannerService; - -#[async_trait] -impl CompactionPlannerService for StubCompactionPlannerService { - async fn ping(&self, _request: PingRequest) -> CompactionResult { - Ok(PingResponse {}) - } - - async fn report_status( - &self, - _request: ReportStatusRequest, - ) -> CompactionResult { - Ok(ReportStatusResponse {}) - } -} diff --git a/quickwit/quickwit-compaction/src/planner/mod.rs b/quickwit/quickwit-compaction/src/planner/mod.rs index e780b30e0ef..b89d67ee8a2 100644 --- a/quickwit/quickwit-compaction/src/planner/mod.rs +++ b/quickwit/quickwit-compaction/src/planner/mod.rs @@ -12,13 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#[allow(dead_code)] mod compaction_planner; -mod compaction_service; #[allow(dead_code)] mod compaction_state; #[allow(dead_code)] mod index_config_store; pub use compaction_planner::CompactionPlanner; -pub use compaction_service::StubCompactionPlannerService; diff --git a/quickwit/quickwit-proto/protos/quickwit/compaction.proto b/quickwit/quickwit-proto/protos/quickwit/compaction.proto index f071cc3bc01..7313d85db91 100644 --- a/quickwit/quickwit-proto/protos/quickwit/compaction.proto +++ b/quickwit/quickwit-proto/protos/quickwit/compaction.proto @@ -19,13 +19,9 @@ package quickwit.compaction; import "quickwit/common.proto"; service CompactionPlannerService { - rpc Ping(PingRequest) returns (PingResponse); rpc ReportStatus(ReportStatusRequest) returns (ReportStatusResponse); } -message PingRequest {} -message PingResponse {} - message ReportStatusRequest { string node_id = 1; uint32 available_slots = 2; @@ -51,4 +47,18 @@ message CompactionFailure { string error_message = 2; } -message ReportStatusResponse {} \ No newline at end of file +message ReportStatusResponse { + repeated MergeTaskAssignment new_tasks = 1; +} + +message MergeTaskAssignment { + string task_id = 1; + repeated string splits_metadata_json = 2; + string doc_mapping_json = 3; + string search_settings_json = 4; + string indexing_settings_json = 5; + string retention_policy_json = 6; + quickwit.common.IndexUid index_uid = 7; + string source_id = 8; + string index_storage_uri = 9; +} \ No newline at end of file diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs index df33d068f9a..f96bb33ed20 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs @@ -1,11 +1,5 @@ // This file is @generated by prost-build. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] -pub struct PingRequest {} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] -pub struct PingResponse {} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReportStatusRequest { #[prost(string, tag = "1")] @@ -48,18 +42,38 @@ pub struct CompactionFailure { pub error_message: ::prost::alloc::string::String, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[derive(Clone, Copy, PartialEq, Eq, Hash, ::prost::Message)] -pub struct ReportStatusResponse {} +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReportStatusResponse { + #[prost(message, repeated, tag = "1")] + pub new_tasks: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct MergeTaskAssignment { + #[prost(string, tag = "1")] + pub task_id: ::prost::alloc::string::String, + #[prost(string, repeated, tag = "2")] + pub splits_metadata_json: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(string, tag = "3")] + pub doc_mapping_json: ::prost::alloc::string::String, + #[prost(string, tag = "4")] + pub search_settings_json: ::prost::alloc::string::String, + #[prost(string, tag = "5")] + pub indexing_settings_json: ::prost::alloc::string::String, + #[prost(string, tag = "6")] + pub retention_policy_json: ::prost::alloc::string::String, + #[prost(message, optional, tag = "7")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "8")] + pub source_id: ::prost::alloc::string::String, + #[prost(string, tag = "9")] + pub index_storage_uri: ::prost::alloc::string::String, +} /// BEGIN quickwit-codegen #[allow(unused_imports)] use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; use quickwit_common::tower::RpcName; -impl RpcName for PingRequest { - fn rpc_name() -> &'static str { - "ping" - } -} impl RpcName for ReportStatusRequest { fn rpc_name() -> &'static str { "report_status" @@ -68,10 +82,6 @@ impl RpcName for ReportStatusRequest { #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] pub trait CompactionPlannerService: std::fmt::Debug + Send + Sync + 'static { - async fn ping( - &self, - request: PingRequest, - ) -> crate::compaction::CompactionResult; async fn report_status( &self, request: ReportStatusRequest, @@ -188,12 +198,6 @@ impl CompactionPlannerServiceClient { } #[async_trait::async_trait] impl CompactionPlannerService for CompactionPlannerServiceClient { - async fn ping( - &self, - request: PingRequest, - ) -> crate::compaction::CompactionResult { - self.inner.0.ping(request).await - } async fn report_status( &self, request: ReportStatusRequest, @@ -210,12 +214,6 @@ pub mod mock_compaction_planner_service { } #[async_trait::async_trait] impl CompactionPlannerService for MockCompactionPlannerServiceWrapper { - async fn ping( - &self, - request: super::PingRequest, - ) -> crate::compaction::CompactionResult { - self.inner.lock().await.ping(request).await - } async fn report_status( &self, request: super::ReportStatusRequest, @@ -227,22 +225,6 @@ pub mod mock_compaction_planner_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for InnerCompactionPlannerServiceClient { - type Response = PingResponse; - type Error = crate::compaction::CompactionError; - type Future = BoxFuture; - fn poll_ready( - &mut self, - _cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - std::task::Poll::Ready(Ok(())) - } - fn call(&mut self, request: PingRequest) -> Self::Future { - let svc = self.clone(); - let fut = async move { svc.0.ping(request).await }; - Box::pin(fut) - } -} impl tower::Service for InnerCompactionPlannerServiceClient { type Response = ReportStatusResponse; type Error = crate::compaction::CompactionError; @@ -264,11 +246,6 @@ impl tower::Service for InnerCompactionPlannerServiceClient struct CompactionPlannerServiceTowerServiceStack { #[allow(dead_code)] inner: InnerCompactionPlannerServiceClient, - ping_svc: quickwit_common::tower::BoxService< - PingRequest, - PingResponse, - crate::compaction::CompactionError, - >, report_status_svc: quickwit_common::tower::BoxService< ReportStatusRequest, ReportStatusResponse, @@ -277,12 +254,6 @@ struct CompactionPlannerServiceTowerServiceStack { } #[async_trait::async_trait] impl CompactionPlannerService for CompactionPlannerServiceTowerServiceStack { - async fn ping( - &self, - request: PingRequest, - ) -> crate::compaction::CompactionResult { - self.ping_svc.clone().ready().await?.call(request).await - } async fn report_status( &self, request: ReportStatusRequest, @@ -290,16 +261,6 @@ impl CompactionPlannerService for CompactionPlannerServiceTowerServiceStack { self.report_status_svc.clone().ready().await?.call(request).await } } -type PingLayer = quickwit_common::tower::BoxLayer< - quickwit_common::tower::BoxService< - PingRequest, - PingResponse, - crate::compaction::CompactionError, - >, - PingRequest, - PingResponse, - crate::compaction::CompactionError, ->; type ReportStatusLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< ReportStatusRequest, @@ -312,37 +273,11 @@ type ReportStatusLayer = quickwit_common::tower::BoxLayer< >; #[derive(Debug, Default)] pub struct CompactionPlannerServiceTowerLayerStack { - ping_layers: Vec, report_status_layers: Vec, } impl CompactionPlannerServiceTowerLayerStack { pub fn stack_layer(mut self, layer: L) -> Self where - L: tower::Layer< - quickwit_common::tower::BoxService< - PingRequest, - PingResponse, - crate::compaction::CompactionError, - >, - > + Clone + Send + Sync + 'static, - , - >>::Service: tower::Service< - PingRequest, - Response = PingResponse, - Error = crate::compaction::CompactionError, - > + Clone + Send + Sync + 'static, - <, - >>::Service as tower::Service>::Future: Send + 'static, L: tower::Layer< quickwit_common::tower::BoxService< ReportStatusRequest, @@ -369,30 +304,10 @@ impl CompactionPlannerServiceTowerLayerStack { >, >>::Service as tower::Service>::Future: Send + 'static, { - self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.report_status_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self } - pub fn stack_ping_layer(mut self, layer: L) -> Self - where - L: tower::Layer< - quickwit_common::tower::BoxService< - PingRequest, - PingResponse, - crate::compaction::CompactionError, - >, - > + Send + Sync + 'static, - L::Service: tower::Service< - PingRequest, - Response = PingResponse, - Error = crate::compaction::CompactionError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, - { - self.ping_layers.push(quickwit_common::tower::BoxLayer::new(layer)); - self - } pub fn stack_report_status_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -477,14 +392,6 @@ impl CompactionPlannerServiceTowerLayerStack { self, inner_client: InnerCompactionPlannerServiceClient, ) -> CompactionPlannerServiceClient { - let ping_svc = self - .ping_layers - .into_iter() - .rev() - .fold( - quickwit_common::tower::BoxService::new(inner_client.clone()), - |svc, layer| layer.layer(svc), - ); let report_status_svc = self .report_status_layers .into_iter() @@ -495,7 +402,6 @@ impl CompactionPlannerServiceTowerLayerStack { ); let tower_svc_stack = CompactionPlannerServiceTowerServiceStack { inner: inner_client, - ping_svc, report_status_svc, }; CompactionPlannerServiceClient::new(tower_svc_stack) @@ -574,24 +480,12 @@ where CompactionPlannerServiceMailbox< A, >: tower::Service< - PingRequest, - Response = PingResponse, - Error = crate::compaction::CompactionError, - Future = BoxFuture, - > - + tower::Service< - ReportStatusRequest, - Response = ReportStatusResponse, - Error = crate::compaction::CompactionError, - Future = BoxFuture, - >, + ReportStatusRequest, + Response = ReportStatusResponse, + Error = crate::compaction::CompactionError, + Future = BoxFuture, + >, { - async fn ping( - &self, - request: PingRequest, - ) -> crate::compaction::CompactionResult { - self.clone().call(request).await - } async fn report_status( &self, request: ReportStatusRequest, @@ -633,20 +527,6 @@ where + Send, T::Future: Send, { - async fn ping( - &self, - request: PingRequest, - ) -> crate::compaction::CompactionResult { - self.inner - .clone() - .ping(request) - .await - .map(|response| response.into_inner()) - .map_err(|status| crate::error::grpc_status_to_service_error( - status, - PingRequest::rpc_name(), - )) - } async fn report_status( &self, request: ReportStatusRequest, @@ -679,17 +559,6 @@ impl CompactionPlannerServiceGrpcServerAdapter { #[async_trait::async_trait] impl compaction_planner_service_grpc_server::CompactionPlannerServiceGrpc for CompactionPlannerServiceGrpcServerAdapter { - async fn ping( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - self.inner - .0 - .ping(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(crate::error::grpc_error_to_grpc_status) - } async fn report_status( &self, request: tonic::Request, @@ -795,32 +664,6 @@ pub mod compaction_planner_service_grpc_client { self.inner = self.inner.max_encoding_message_size(limit); self } - pub async fn ping( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::unknown( - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic_prost::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.compaction.CompactionPlannerService/Ping", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.compaction.CompactionPlannerService", - "Ping", - ), - ); - self.inner.unary(req, path, codec).await - } pub async fn report_status( &mut self, request: impl tonic::IntoRequest, @@ -865,10 +708,6 @@ pub mod compaction_planner_service_grpc_server { /// Generated trait containing gRPC methods that should be implemented for use with CompactionPlannerServiceGrpcServer. #[async_trait] pub trait CompactionPlannerServiceGrpc: std::marker::Send + std::marker::Sync + 'static { - async fn ping( - &self, - request: tonic::Request, - ) -> std::result::Result, tonic::Status>; async fn report_status( &self, request: tonic::Request, @@ -954,51 +793,6 @@ pub mod compaction_planner_service_grpc_server { } fn call(&mut self, req: http::Request) -> Self::Future { match req.uri().path() { - "/quickwit.compaction.CompactionPlannerService/Ping" => { - #[allow(non_camel_case_types)] - struct PingSvc(pub Arc); - impl< - T: CompactionPlannerServiceGrpc, - > tonic::server::UnaryService for PingSvc { - type Response = super::PingResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; - fn call( - &mut self, - request: tonic::Request, - ) -> Self::Future { - let inner = Arc::clone(&self.0); - let fut = async move { - ::ping(&inner, request) - .await - }; - Box::pin(fut) - } - } - let accept_compression_encodings = self.accept_compression_encodings; - let send_compression_encodings = self.send_compression_encodings; - let max_decoding_message_size = self.max_decoding_message_size; - let max_encoding_message_size = self.max_encoding_message_size; - let inner = self.inner.clone(); - let fut = async move { - let method = PingSvc(inner); - let codec = tonic_prost::ProstCodec::default(); - let mut grpc = tonic::server::Grpc::new(codec) - .apply_compression_config( - accept_compression_encodings, - send_compression_encodings, - ) - .apply_max_message_size_config( - max_decoding_message_size, - max_encoding_message_size, - ); - let res = grpc.unary(method, req).await; - Ok(res) - }; - Box::pin(fut) - } "/quickwit.compaction.CompactionPlannerService/ReportStatus" => { #[allow(non_camel_case_types)] struct ReportStatusSvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/compaction/mod.rs b/quickwit/quickwit-proto/src/compaction/mod.rs index 1e27855add3..0eee7adb6b0 100644 --- a/quickwit/quickwit-proto/src/compaction/mod.rs +++ b/quickwit/quickwit-proto/src/compaction/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use quickwit_actors::AskError; use quickwit_common::rate_limited_error; use quickwit_common::tower::MakeLoadShedError; use serde::{Deserialize, Serialize}; @@ -74,3 +75,17 @@ impl MakeLoadShedError for CompactionError { CompactionError::TooManyRequests } } + +impl From> for CompactionError { + fn from(error: AskError) -> Self { + match error { + AskError::ErrorReply(error) => error, + AskError::MessageNotDelivered => { + Self::new_unavailable("request could not be delivered to actor".to_string()) + } + AskError::ProcessMessageError => { + Self::new_internal("an error occurred while processing the request".to_string()) + } + } + } +} diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 9dafbfe9d4a..bc6db8835e0 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -72,7 +72,7 @@ use quickwit_common::tower::{ }; use quickwit_common::uri::Uri; use quickwit_common::{get_bool_from_env, spawn_named_task}; -use quickwit_compaction::planner::StubCompactionPlannerService; +use quickwit_compaction::planner::CompactionPlanner; use quickwit_compaction::{CompactorSupervisor, start_compactor_service}; use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IngestApiConfig, NodeConfig}; @@ -271,23 +271,28 @@ async fn balance_channel_for_service( BalanceChannel::from_stream(service_change_stream) } -/// Builds a `CompactionPlannerServiceClient` if the node runs the compactor. +/// Builds a `CompactionPlannerServiceClient` if the node runs the janitor or compactor. /// -/// On janitor+compactor nodes, wraps a local `StubCompactionPlannerService`. -/// On compactor-only nodes, connects to a remote janitor via gRPC. +/// On janitor nodes, spawns a `CompactionPlanner` actor and builds the client from +/// its mailbox. On compactor-only nodes, connects to a remote janitor via gRPC. async fn get_compaction_planner_client_if_needed( node_config: &NodeConfig, cluster: &Cluster, -) -> anyhow::Result, anyhow::Error> { - if !node_config.is_service_enabled(QuickwitService::Compactor) { + universe: &Universe, + metastore_client: &MetastoreServiceClient, +) -> anyhow::Result> { + let is_janitor = node_config.is_service_enabled(QuickwitService::Janitor); + let is_compactor = node_config.is_service_enabled(QuickwitService::Compactor); + if !is_janitor && !is_compactor { return Ok(None); } - if node_config.is_service_enabled(QuickwitService::Janitor) { - info!("compaction planner service enabled on this node"); - return Ok(Some(CompactionPlannerServiceClient::new( - StubCompactionPlannerService, - ))); + if is_janitor { + let planner = CompactionPlanner::new(metastore_client.clone()); + let (mailbox, _handle) = universe.spawn_builder().spawn(planner); + info!("compaction planner actor started on janitor node"); + return Ok(Some(CompactionPlannerServiceClient::from_mailbox(mailbox))); } + // Compactor-only node: connect to the planner on a remote janitor. let balance_channel = balance_channel_for_service(cluster, QuickwitService::Janitor).await; let found = balance_channel .wait_for(COMPACTION_SERVICE_DISCOVERY_TIMEOUT, |connections| { @@ -593,10 +598,14 @@ pub async fn serve_quickwit( .await .context("failed to start ingest v1 service")?; - let compaction_service_client_opt = - get_compaction_planner_client_if_needed(&node_config, &cluster) - .await - .context("failed to initialize compaction service client")?; + let compaction_service_client_opt = get_compaction_planner_client_if_needed( + &node_config, + &cluster, + &universe, + &metastore_client, + ) + .await + .context("failed to initialize compaction service client")?; let indexing_service_opt = if node_config.is_service_enabled(QuickwitService::Indexer) { let merge_scheduler_mailbox_opt = @@ -1917,26 +1926,40 @@ mod tests { create_cluster_for_test(Vec::new(), &["janitor", "indexer"], &transport, true) .await .unwrap(); + let universe = Universe::new(); + let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); - // Without compactor service enabled, no planner client. + // Janitor without compactor: planner client is returned (for gRPC registration). let mut node_config = NodeConfig::for_test(); node_config.enabled_services = HashSet::from([QuickwitService::Janitor, QuickwitService::Indexer]); - let result = get_compaction_planner_client_if_needed(&node_config, &cluster) - .await - .unwrap(); - assert!(result.is_none()); + let result = + get_compaction_planner_client_if_needed(&node_config, &cluster, &universe, &metastore) + .await + .unwrap(); + assert!(result.is_some()); - // With compactor + janitor enabled, planner client is returned (local stub). + // With compactor + janitor enabled, planner client is also returned. node_config.enabled_services = HashSet::from([ QuickwitService::Janitor, QuickwitService::Indexer, QuickwitService::Compactor, ]); - let result = get_compaction_planner_client_if_needed(&node_config, &cluster) - .await - .unwrap(); + let result = + get_compaction_planner_client_if_needed(&node_config, &cluster, &universe, &metastore) + .await + .unwrap(); assert!(result.is_some()); + + // Neither janitor nor compactor: no client. + node_config.enabled_services = HashSet::from([QuickwitService::Indexer]); + let result = + get_compaction_planner_client_if_needed(&node_config, &cluster, &universe, &metastore) + .await + .unwrap(); + assert!(result.is_none()); + + universe.assert_quit().await; } #[tokio::test] @@ -1945,11 +1968,17 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, false) .await .unwrap(); + let universe = Universe::new(); + let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); let mut node_config = NodeConfig::for_test(); node_config.enabled_services = HashSet::from([QuickwitService::Indexer, QuickwitService::Compactor]); - let result = get_compaction_planner_client_if_needed(&node_config, &cluster).await; + let result = + get_compaction_planner_client_if_needed(&node_config, &cluster, &universe, &metastore) + .await; assert!(result.is_err()); + + universe.assert_quit().await; } } From 48d499c4fb88bc8ab752d5335812528c2fb56c83 Mon Sep 17 00:00:00 2001 From: nadav-govari Date: Thu, 16 Apr 2026 15:09:15 -0400 Subject: [PATCH 09/11] Make ReportStatus gRPC from compactors (#6311) --- .../src/compaction_pipeline.rs | 4 + .../src/compactor_supervisor.rs | 306 +++++++++++++++++- quickwit/quickwit-compaction/src/lib.rs | 8 +- quickwit/quickwit-indexing/src/lib.rs | 4 +- quickwit/quickwit-serve/src/lib.rs | 11 +- 5 files changed, 308 insertions(+), 25 deletions(-) diff --git a/quickwit/quickwit-compaction/src/compaction_pipeline.rs b/quickwit/quickwit-compaction/src/compaction_pipeline.rs index 62bb040e0ba..021b3fa27c5 100644 --- a/quickwit/quickwit-compaction/src/compaction_pipeline.rs +++ b/quickwit/quickwit-compaction/src/compaction_pipeline.rs @@ -113,6 +113,10 @@ impl CompactionPipeline { } } + pub fn status(&self) -> &PipelineStatus { + &self.status + } + fn supervisables(&self) -> Vec<&dyn Supervisable> { let Some(handles) = &self.handles else { return Vec::new(); diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index e1b27eb4a30..292c861972c 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -12,22 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::str::FromStr; +use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; -use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; +use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, SpawnContext}; use quickwit_common::io::Limiter; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; -use quickwit_indexing::IndexingSplitStore; +use quickwit_common::uri::Uri; +use quickwit_config::{IndexingSettings, RetentionPolicy, SearchSettings, build_doc_mapper}; +use quickwit_doc_mapper::DocMapping; +use quickwit_indexing::merge_policy::{MergeOperation, merge_policy_from_settings}; +use quickwit_indexing::{IndexingSplitCache, IndexingSplitStore}; +use quickwit_metastore::SplitMetadata; use quickwit_proto::compaction::{ - CompactionFailure, CompactionInProgress, CompactionPlannerServiceClient, CompactionSuccess, - ReportStatusRequest, + CompactionFailure, CompactionInProgress, CompactionPlannerService, + CompactionPlannerServiceClient, CompactionSuccess, MergeTaskAssignment, ReportStatusRequest, }; +use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::types::NodeId; use quickwit_storage::StorageResolver; -use tracing::info; +use tracing::{error, info, warn}; use crate::compaction_pipeline::{CompactionPipeline, PipelineStatus, PipelineStatusUpdate}; @@ -47,7 +55,7 @@ pub struct CompactorSupervisor { // Shared resources distributed to pipelines when spawning actor chains. io_throughput_limiter: Option, - split_store: IndexingSplitStore, + split_cache: Arc, metastore: MetastoreServiceClient, storage_resolver: StorageResolver, max_concurrent_split_uploads: usize, @@ -64,7 +72,7 @@ impl CompactorSupervisor { planner_client: CompactionPlannerServiceClient, num_pipeline_slots: usize, io_throughput_limiter: Option, - split_store: IndexingSplitStore, + split_cache: Arc, metastore: MetastoreServiceClient, storage_resolver: StorageResolver, max_concurrent_split_uploads: usize, @@ -77,7 +85,7 @@ impl CompactorSupervisor { planner_client, pipelines, io_throughput_limiter, - split_store, + split_cache, metastore, storage_resolver, max_concurrent_split_uploads, @@ -97,6 +105,100 @@ impl CompactorSupervisor { statuses } + async fn process_new_tasks( + &mut self, + assignments: Vec, + spawn_ctx: &SpawnContext, + ) { + for assignment in assignments { + let task_id = assignment.task_id.clone(); + if let Err(err) = self.spawn_task(assignment, spawn_ctx).await { + error!(task_id=%task_id, error=%err, "failed to spawn compaction task"); + } + } + } + + async fn spawn_task( + &mut self, + assignment: MergeTaskAssignment, + spawn_ctx: &SpawnContext, + ) -> anyhow::Result<()> { + let slot_idx = self + .pipelines + .iter() + .position(|slot| match slot { + None => true, + Some(p) => matches!( + p.status(), + PipelineStatus::Completed | PipelineStatus::Failed { .. } + ), + }) + .ok_or_else(|| anyhow::anyhow!("no free pipeline slot"))?; + let scratch_directory = self + .compaction_root_directory + .named_temp_child(&assignment.task_id)?; + let mut pipeline = self + .build_compaction_pipeline(assignment, scratch_directory) + .await?; + pipeline.spawn_pipeline(spawn_ctx)?; + self.pipelines[slot_idx] = Some(pipeline); + Ok(()) + } + + async fn build_compaction_pipeline( + &self, + assignment: MergeTaskAssignment, + scratch_directory: TempDirectory, + ) -> anyhow::Result { + let splits: Vec = assignment + .splits_metadata_json + .iter() + .map(|json| serde_json::from_str(json)) + .collect::, serde_json::Error>>()?; + let doc_mapping: DocMapping = serde_json::from_str(&assignment.doc_mapping_json)?; + let search_settings: SearchSettings = + serde_json::from_str(&assignment.search_settings_json)?; + let indexing_settings: IndexingSettings = + serde_json::from_str(&assignment.indexing_settings_json)?; + let retention_policy: Option = + if assignment.retention_policy_json.is_empty() { + None + } else { + Some(serde_json::from_str(&assignment.retention_policy_json)?) + }; + let index_uid = assignment + .index_uid + .ok_or_else(|| anyhow::anyhow!("missing index_uid in MergeTaskAssignment"))?; + + let index_storage_uri = Uri::from_str(&assignment.index_storage_uri)?; + let index_storage = self.storage_resolver.resolve(&index_storage_uri).await?; + let split_store = IndexingSplitStore::new(index_storage, self.split_cache.clone()); + + let doc_mapper = build_doc_mapper(&doc_mapping, &search_settings)?; + let merge_policy = merge_policy_from_settings(&indexing_settings); + let merge_operation = MergeOperation::new_merge_operation(splits); + let pipeline_id = MergePipelineId { + node_id: self.node_id.clone(), + index_uid, + source_id: assignment.source_id, + }; + + Ok(CompactionPipeline::new( + assignment.task_id, + scratch_directory, + merge_operation, + pipeline_id, + doc_mapper, + merge_policy, + retention_policy, + self.metastore.clone(), + split_store, + self.io_throughput_limiter.clone(), + self.max_concurrent_split_uploads, + self.event_broker.clone(), + )) + } + fn build_report_status_request( &self, statuses: &[PipelineStatusUpdate], @@ -176,8 +278,16 @@ impl Handler for CompactorSupervisor { ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { let statuses = self.check_pipeline_statuses(); - let _request = self.build_report_status_request(&statuses); - // TODO: send request to planner via gRPC, clear completed/failed slots on success. + let request = self.build_report_status_request(&statuses); + match self.planner_client.report_status(request).await { + Ok(response) => { + self.process_new_tasks(response.new_tasks, ctx.spawn_ctx()) + .await; + } + Err(err) => { + warn!(error=%err, "failed to report status to compaction planner"); + } + } ctx.schedule_self_msg(CHECK_PIPELINE_STATUSES_INTERVAL, CheckPipelineStatuses); Ok(()) } @@ -194,14 +304,12 @@ mod tests { }; use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; use quickwit_proto::types::NodeId; - use quickwit_storage::{RamStorage, StorageResolver}; + use quickwit_storage::StorageResolver; use super::*; use crate::compaction_pipeline::tests::test_pipeline; fn test_supervisor(num_slots: usize) -> CompactorSupervisor { - let storage = Arc::new(RamStorage::default()); - let split_store = IndexingSplitStore::create_without_local_store_for_test(storage); let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); let compaction_client = CompactionPlannerServiceClient::from_mock(MockCompactionPlannerService::new()); @@ -210,7 +318,7 @@ mod tests { compaction_client, num_slots, None, - split_store, + Arc::new(IndexingSplitCache::no_caching()), metastore, StorageResolver::for_test(), 2, @@ -282,6 +390,176 @@ mod tests { assert!(request.failures.is_empty()); } + fn test_assignment(task_id: &str, split_ids: &[&str]) -> MergeTaskAssignment { + let index_metadata = + quickwit_metastore::IndexMetadata::for_test("test-index", "ram:///test-index"); + let config = &index_metadata.index_config; + let splits: Vec = split_ids + .iter() + .map(|id| quickwit_metastore::SplitMetadata::for_test(id.to_string())) + .collect(); + MergeTaskAssignment { + task_id: task_id.to_string(), + splits_metadata_json: splits + .iter() + .map(|s| serde_json::to_string(s).unwrap()) + .collect(), + doc_mapping_json: serde_json::to_string(&config.doc_mapping).unwrap(), + search_settings_json: serde_json::to_string(&config.search_settings).unwrap(), + indexing_settings_json: serde_json::to_string(&config.indexing_settings).unwrap(), + retention_policy_json: String::new(), + index_uid: Some(index_metadata.index_uid.clone()), + source_id: "test-source".to_string(), + index_storage_uri: config.index_uri.to_string(), + } + } + + #[tokio::test] + async fn test_build_compaction_pipeline_deserialization_errors() { + let supervisor = test_supervisor(4); + let scratch = TempDirectory::for_test; + + // Bad splits JSON. + let mut assignment = test_assignment("t", &["s1"]); + assignment.splits_metadata_json = vec!["not json".to_string()]; + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + + // Bad doc mapping JSON. + let mut assignment = test_assignment("t", &["s1"]); + assignment.doc_mapping_json = "not json".to_string(); + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + + // Bad search settings JSON. + let mut assignment = test_assignment("t", &["s1"]); + assignment.search_settings_json = "not json".to_string(); + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + + // Bad indexing settings JSON. + let mut assignment = test_assignment("t", &["s1"]); + assignment.indexing_settings_json = "not json".to_string(); + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + + // Bad retention policy JSON (non-empty but invalid). + let mut assignment = test_assignment("t", &["s1"]); + assignment.retention_policy_json = "not json".to_string(); + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + + // Missing index_uid. + let mut assignment = test_assignment("t", &["s1"]); + assignment.index_uid = None; + assert!( + supervisor + .build_compaction_pipeline(assignment, scratch()) + .await + .is_err() + ); + } + + #[tokio::test] + async fn test_spawn_task_fails_when_all_slots_occupied() { + let universe = Universe::new(); + let mut supervisor = test_supervisor(2); + + supervisor + .spawn_task(test_assignment("task-1", &["s1"]), universe.spawn_ctx()) + .await + .unwrap(); + supervisor + .spawn_task(test_assignment("task-2", &["s2"]), universe.spawn_ctx()) + .await + .unwrap(); + + // Both slots InProgress — no room. + assert!( + supervisor + .spawn_task(test_assignment("task-3", &["s3"]), universe.spawn_ctx()) + .await + .is_err() + ); + + universe.assert_quit().await; + } + + #[tokio::test] + async fn test_end_to_end_report_status_and_spawn() { + let universe = Universe::new(); + + // Mock planner that returns one assignment on the first call. + let assignment = test_assignment("planner-task-1", &["s1", "s2"]); + let assignments = vec![assignment]; + let assignments_clone = assignments.clone(); + let mut mock = quickwit_proto::compaction::MockCompactionPlannerService::new(); + mock.expect_report_status().times(1).returning(move |_req| { + Ok(quickwit_proto::compaction::ReportStatusResponse { + new_tasks: assignments_clone.clone(), + }) + }); + + let metastore = MetastoreServiceClient::from_mock(MockMetastoreService::new()); + let client = CompactionPlannerServiceClient::from_mock(mock); + let mut supervisor = CompactorSupervisor::new( + NodeId::from("test-node"), + client, + 3, + None, + Arc::new(IndexingSplitCache::no_caching()), + metastore, + StorageResolver::for_test(), + 2, + EventBroker::default(), + TempDirectory::for_test(), + ); + + // Simulate what the handler does: collect statuses, report, process response. + let statuses = supervisor.check_pipeline_statuses(); + let request = supervisor.build_report_status_request(&statuses); + assert_eq!(request.available_slots, 3); + + let response = supervisor + .planner_client + .report_status(request) + .await + .unwrap(); + supervisor + .process_new_tasks(response.new_tasks, universe.spawn_ctx()) + .await; + + // Verify the pipeline was spawned. + let statuses = supervisor.check_pipeline_statuses(); + let request = supervisor.build_report_status_request(&statuses); + assert_eq!(request.in_progress.len(), 1); + assert_eq!(request.in_progress[0].task_id, "planner-task-1"); + assert_eq!(request.in_progress[0].split_ids.len(), 2); + assert_eq!(request.available_slots, 2); + + universe.assert_quit().await; + } + #[test] fn test_build_report_status_request_mixed_statuses() { let supervisor = test_supervisor(4); diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs index 05f79d83f73..98e5fc9a3d1 100644 --- a/quickwit/quickwit-compaction/src/lib.rs +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -20,13 +20,15 @@ mod compaction_pipeline; mod compactor_supervisor; pub mod planner; +use std::sync::Arc; + pub use compactor_supervisor::CompactorSupervisor; use quickwit_actors::{Mailbox, Universe}; use quickwit_common::io; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_config::CompactorConfig; -use quickwit_indexing::IndexingSplitStore; +use quickwit_indexing::IndexingSplitCache; use quickwit_proto::compaction::CompactionPlannerServiceClient; use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::types::NodeId; @@ -39,7 +41,7 @@ pub async fn start_compactor_service( node_id: NodeId, compaction_client: CompactionPlannerServiceClient, compactor_config: &CompactorConfig, - split_store: IndexingSplitStore, + split_cache: Arc, metastore: MetastoreServiceClient, storage_resolver: StorageResolver, event_broker: EventBroker, @@ -53,7 +55,7 @@ pub async fn start_compactor_service( compaction_client, compactor_config.max_concurrent_pipelines.get(), io_throughput_limiter, - split_store, + split_cache, metastore, storage_resolver, compactor_config.max_concurrent_split_uploads, diff --git a/quickwit/quickwit-indexing/src/lib.rs b/quickwit/quickwit-indexing/src/lib.rs index 8adde285bf3..66c50d4d150 100644 --- a/quickwit/quickwit-indexing/src/lib.rs +++ b/quickwit/quickwit-indexing/src/lib.rs @@ -31,7 +31,9 @@ pub use crate::actors::{ }; pub use crate::controlled_directory::ControlledDirectory; use crate::models::IndexingStatistics; -pub use crate::split_store::{IndexingSplitStore, get_tantivy_directory_from_split_bundle}; +pub use crate::split_store::{ + IndexingSplitCache, IndexingSplitStore, get_tantivy_directory_from_split_bundle, +}; pub mod actors; mod controlled_directory; diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index bc6db8835e0..4c2e8d8499c 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -81,7 +81,7 @@ use quickwit_control_plane::{IndexerNodeInfo, IndexerPool}; use quickwit_index_management::{IndexService as IndexManager, IndexServiceError}; use quickwit_indexing::actors::{IndexingService, MergeSchedulerService}; use quickwit_indexing::models::ShardPositionsService; -use quickwit_indexing::{IndexingSplitStore, start_indexing_service}; +use quickwit_indexing::start_indexing_service; use quickwit_ingest::{ GetMemoryCapacity, IngestRequest, IngestRouter, IngestServiceClient, Ingester, IngesterPool, IngesterPoolEntry, LocalShardsUpdate, get_idle_shard_timeout, @@ -114,7 +114,7 @@ use quickwit_search::{ SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, create_search_client_from_channel, start_searcher_service, }; -use quickwit_storage::{RamStorage, SplitCache, StorageResolver}; +use quickwit_storage::{SplitCache, StorageResolver}; use tcp_listener::TcpListenerResolver; use tokio::sync::oneshot; use tonic::codec::CompressionEncoding; @@ -795,10 +795,7 @@ pub async fn serve_quickwit( let compaction_root_directory = quickwit_common::temp_dir::Builder::default() .tempdir_in(&compaction_dir) .context("failed to create compaction temp directory")?; - // TODO: Real split store - let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new( - RamStorage::default(), - )); + let split_cache = Arc::new(quickwit_indexing::IndexingSplitCache::no_caching()); let compaction_client = compaction_service_client_opt .clone() .expect("compactor service enabled but no compaction client available"); @@ -807,7 +804,7 @@ pub async fn serve_quickwit( cluster.self_node_id().into(), compaction_client, &node_config.compactor_config, - split_store, + split_cache, metastore_client.clone(), storage_resolver.clone(), event_broker.clone(), From 003eda5c2e1c54cdca65e5e943f4fcbfc249edb6 Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Wed, 6 May 2026 11:19:19 -0400 Subject: [PATCH 10/11] parquet compaction in compaction service --- quickwit/Cargo.lock | 1 + quickwit/quickwit-cli/Cargo.toml | 4 +- quickwit/quickwit-compaction/Cargo.toml | 9 + .../src/compaction_pipeline.rs | 3 + .../src/compactor_supervisor.rs | 111 ++++- quickwit/quickwit-compaction/src/lib.rs | 3 + .../quickwit-compaction/src/parquet/mod.rs | 20 + .../src/parquet/pipeline.rs | 458 +++++++++++++++++ .../src/parquet/planner.rs | 469 ++++++++++++++++++ .../quickwit-compaction/src/parquet/state.rs | 277 +++++++++++ .../src/planner/compaction_planner.rs | 74 ++- .../src/planner/compaction_state.rs | 4 + .../quickwit-metastore/src/metastore/mod.rs | 12 + .../protos/quickwit/compaction.proto | 11 +- .../codegen/quickwit/quickwit.compaction.rs | 36 ++ quickwit/quickwit-serve/Cargo.toml | 4 + 16 files changed, 1480 insertions(+), 16 deletions(-) create mode 100644 quickwit/quickwit-compaction/src/parquet/mod.rs create mode 100644 quickwit/quickwit-compaction/src/parquet/pipeline.rs create mode 100644 quickwit/quickwit-compaction/src/parquet/planner.rs create mode 100644 quickwit/quickwit-compaction/src/parquet/state.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 0b5b22374e0..aa29d0bae7d 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -8371,6 +8371,7 @@ dependencies = [ "quickwit-doc-mapper", "quickwit-indexing", "quickwit-metastore", + "quickwit-parquet-engine", "quickwit-proto", "quickwit-storage", "serde_json", diff --git a/quickwit/quickwit-cli/Cargo.toml b/quickwit/quickwit-cli/Cargo.toml index ebf3ae90e8c..370f7c40f74 100644 --- a/quickwit/quickwit-cli/Cargo.toml +++ b/quickwit/quickwit-cli/Cargo.toml @@ -98,7 +98,7 @@ pprof = ["quickwit-serve/pprof"] openssl-support = ["openssl-probe"] # metrics here refers to adding support for metrics ingestion within Quickwit. # (this is not about quickwit's metrics themselves) -metrics = [ "quickwit-indexing/metrics" ] +metrics = [ "quickwit-indexing/metrics", "quickwit-serve/metrics" ] # Requires to enable tokio unstable via RUSTFLAGS="--cfg tokio_unstable" tokio-console = ["console-subscriber", "quickwit-common/named_tasks"] release-feature-set = [ @@ -125,6 +125,7 @@ release-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka", + "quickwit-serve/metrics", "quickwit-serve/lambda", "quickwit-storage/azure", "quickwit-storage/gcs", @@ -139,6 +140,7 @@ release-macos-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka-macos", + "quickwit-serve/metrics", "quickwit-serve/lambda", "quickwit-storage/azure", "quickwit-storage/gcs", diff --git a/quickwit/quickwit-compaction/Cargo.toml b/quickwit/quickwit-compaction/Cargo.toml index c018327c8c4..1e5db7c9107 100644 --- a/quickwit/quickwit-compaction/Cargo.toml +++ b/quickwit/quickwit-compaction/Cargo.toml @@ -19,6 +19,7 @@ quickwit-config = { workspace = true } quickwit-doc-mapper = { workspace = true } quickwit-indexing = { workspace = true } quickwit-metastore = { workspace = true } +quickwit-parquet-engine = { workspace = true, optional = true } quickwit-proto = { workspace = true } quickwit-storage = { workspace = true } serde_json = { workspace = true } @@ -27,6 +28,14 @@ tracing = { workspace = true } tokio = { workspace = true } ulid = { workspace = true } +[features] +metrics = [ + "dep:quickwit-parquet-engine", + "quickwit-config/metrics", + "quickwit-doc-mapper/metrics", + "quickwit-indexing/metrics", +] + [dev-dependencies] quickwit-actors = { workspace = true, features = ["testsuite"] } quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-compaction/src/compaction_pipeline.rs b/quickwit/quickwit-compaction/src/compaction_pipeline.rs index f1eb2510928..be8af57edb0 100644 --- a/quickwit/quickwit-compaction/src/compaction_pipeline.rs +++ b/quickwit/quickwit-compaction/src/compaction_pipeline.rs @@ -26,6 +26,7 @@ use quickwit_indexing::actors::{ }; use quickwit_indexing::merge_policy::MergeOperation; use quickwit_indexing::{IndexingSplitStore, SplitsUpdateMailbox}; +use quickwit_proto::compaction::CompactionTaskKind; use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::types::{IndexUid, SourceId, SplitId}; @@ -40,6 +41,7 @@ pub enum PipelineStatus { pub struct PipelineStatusUpdate { pub task_id: String, + pub task_kind: CompactionTaskKind, pub index_uid: IndexUid, pub source_id: SourceId, pub split_ids: Vec, @@ -184,6 +186,7 @@ impl CompactionPipeline { fn build_status_update(&self) -> PipelineStatusUpdate { PipelineStatusUpdate { task_id: self.task_id.clone(), + task_kind: CompactionTaskKind::Tantivy, index_uid: self.pipeline_id.index_uid.clone(), source_id: self.pipeline_id.source_id.clone(), split_ids: self diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index 292c861972c..508a361f15d 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -29,7 +29,8 @@ use quickwit_indexing::{IndexingSplitCache, IndexingSplitStore}; use quickwit_metastore::SplitMetadata; use quickwit_proto::compaction::{ CompactionFailure, CompactionInProgress, CompactionPlannerService, - CompactionPlannerServiceClient, CompactionSuccess, MergeTaskAssignment, ReportStatusRequest, + CompactionPlannerServiceClient, CompactionSuccess, CompactionTaskKind, MergeTaskAssignment, + ReportStatusRequest, }; use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::MetastoreServiceClient; @@ -44,6 +45,38 @@ const CHECK_PIPELINE_STATUSES_INTERVAL: Duration = Duration::from_secs(1); #[derive(Debug)] struct CheckPipelineStatuses; +enum RunningCompactionPipeline { + Tantivy(CompactionPipeline), + #[cfg(feature = "metrics")] + Parquet(crate::parquet::ParquetCompactionPipeline), +} + +impl RunningCompactionPipeline { + fn status(&self) -> &PipelineStatus { + match self { + RunningCompactionPipeline::Tantivy(pipeline) => pipeline.status(), + #[cfg(feature = "metrics")] + RunningCompactionPipeline::Parquet(pipeline) => pipeline.status(), + } + } + + fn pipeline_status_update(&mut self) -> PipelineStatusUpdate { + match self { + RunningCompactionPipeline::Tantivy(pipeline) => pipeline.pipeline_status_update(), + #[cfg(feature = "metrics")] + RunningCompactionPipeline::Parquet(pipeline) => pipeline.pipeline_status_update(), + } + } + + fn spawn_pipeline(&mut self, spawn_ctx: &SpawnContext) -> anyhow::Result<()> { + match self { + RunningCompactionPipeline::Tantivy(pipeline) => pipeline.spawn_pipeline(spawn_ctx), + #[cfg(feature = "metrics")] + RunningCompactionPipeline::Parquet(pipeline) => pipeline.spawn_pipeline(spawn_ctx), + } + } +} + /// Manages a pool of `CompactionPipeline`s, each executing a single merge task. /// /// Periodically collects pipeline status updates and forwards them to the @@ -51,7 +84,7 @@ struct CheckPipelineStatuses; pub struct CompactorSupervisor { node_id: NodeId, planner_client: CompactionPlannerServiceClient, - pipelines: Vec>, + pipelines: Vec>, // Shared resources distributed to pipelines when spawning actor chains. io_throughput_limiter: Option, @@ -149,7 +182,19 @@ impl CompactorSupervisor { &self, assignment: MergeTaskAssignment, scratch_directory: TempDirectory, - ) -> anyhow::Result { + ) -> anyhow::Result { + #[cfg(feature = "metrics")] + if is_parquet_task_kind(assignment.task_kind) { + return self + .build_parquet_compaction_pipeline(assignment, scratch_directory) + .await + .map(RunningCompactionPipeline::Parquet); + } + #[cfg(not(feature = "metrics"))] + if is_parquet_task_kind(assignment.task_kind) { + anyhow::bail!("received parquet compaction task but metrics feature is disabled"); + } + let splits: Vec = assignment .splits_metadata_json .iter() @@ -183,7 +228,7 @@ impl CompactorSupervisor { source_id: assignment.source_id, }; - Ok(CompactionPipeline::new( + Ok(RunningCompactionPipeline::Tantivy(CompactionPipeline::new( assignment.task_id, scratch_directory, merge_operation, @@ -196,6 +241,42 @@ impl CompactorSupervisor { self.io_throughput_limiter.clone(), self.max_concurrent_split_uploads, self.event_broker.clone(), + ))) + } + + #[cfg(feature = "metrics")] + async fn build_parquet_compaction_pipeline( + &self, + assignment: MergeTaskAssignment, + scratch_directory: TempDirectory, + ) -> anyhow::Result { + let splits: Vec = assignment + .splits_metadata_json + .iter() + .map(|json| serde_json::from_str(json)) + .collect::, serde_json::Error>>()?; + anyhow::ensure!( + splits.len() >= 2, + "parquet compaction task requires at least two input splits" + ); + let index_uid = assignment + .index_uid + .ok_or_else(|| anyhow::anyhow!("missing index_uid in MergeTaskAssignment"))?; + + let index_storage_uri = Uri::from_str(&assignment.index_storage_uri)?; + let index_storage = self.storage_resolver.resolve(&index_storage_uri).await?; + let merge_operation = + quickwit_parquet_engine::merge::policy::ParquetMergeOperation::new(splits); + + Ok(crate::parquet::ParquetCompactionPipeline::new( + assignment.task_id, + scratch_directory, + merge_operation, + index_uid, + index_storage, + self.metastore.clone(), + self.max_concurrent_split_uploads, + quickwit_parquet_engine::storage::ParquetWriterConfig::default(), )) } @@ -221,18 +302,21 @@ impl CompactorSupervisor { index_uid: Some(update.index_uid.clone()), source_id: update.source_id.clone(), split_ids: update.split_ids.clone(), + task_kind: update.task_kind as i32, }); } PipelineStatus::Completed => { successes.push(CompactionSuccess { task_id: update.task_id.clone(), merged_split_id: update.merged_split_id.clone(), + task_kind: update.task_kind as i32, }); } PipelineStatus::Failed { error } => { failures.push(CompactionFailure { task_id: update.task_id.clone(), error_message: error.clone(), + task_kind: update.task_kind as i32, }); } } @@ -248,6 +332,13 @@ impl CompactorSupervisor { } } +fn is_parquet_task_kind(task_kind: i32) -> bool { + matches!( + CompactionTaskKind::try_from(task_kind), + Ok(CompactionTaskKind::Parquet) + ) +} + #[async_trait] impl Actor for CompactorSupervisor { type ObservableState = (); @@ -300,7 +391,7 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::temp_dir::TempDirectory; use quickwit_proto::compaction::{ - CompactionPlannerServiceClient, MockCompactionPlannerService, + CompactionPlannerServiceClient, CompactionTaskKind, MockCompactionPlannerService, }; use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; use quickwit_proto::types::NodeId; @@ -341,11 +432,11 @@ mod tests { let mut pipeline = test_pipeline("task-1", &["split-a", "split-b"]); pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); - supervisor.pipelines[0] = Some(pipeline); + supervisor.pipelines[0] = Some(RunningCompactionPipeline::Tantivy(pipeline)); let mut pipeline = test_pipeline("task-2", &["split-c"]); pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); - supervisor.pipelines[2] = Some(pipeline); + supervisor.pipelines[2] = Some(RunningCompactionPipeline::Tantivy(pipeline)); let statuses = supervisor.check_pipeline_statuses(); assert_eq!(statuses.len(), 2); @@ -363,7 +454,7 @@ mod tests { let mut pipeline = test_pipeline("task-1", &["s1", "s2"]); pipeline.spawn_pipeline(universe.spawn_ctx()).unwrap(); - supervisor.pipelines[0] = Some(pipeline); + supervisor.pipelines[0] = Some(RunningCompactionPipeline::Tantivy(pipeline)); let statuses = supervisor.check_pipeline_statuses(); let request = supervisor.build_report_status_request(&statuses); @@ -411,6 +502,7 @@ mod tests { index_uid: Some(index_metadata.index_uid.clone()), source_id: "test-source".to_string(), index_storage_uri: config.index_uri.to_string(), + task_kind: CompactionTaskKind::Tantivy as i32, } } @@ -566,6 +658,7 @@ mod tests { let statuses = vec![ PipelineStatusUpdate { task_id: "task-1".to_string(), + task_kind: CompactionTaskKind::Tantivy, index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), source_id: "src".to_string(), split_ids: vec!["s1".to_string(), "s2".to_string()], @@ -574,6 +667,7 @@ mod tests { }, PipelineStatusUpdate { task_id: "task-2".to_string(), + task_kind: CompactionTaskKind::Tantivy, index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), source_id: "src".to_string(), split_ids: vec!["s3".to_string()], @@ -582,6 +676,7 @@ mod tests { }, PipelineStatusUpdate { task_id: "task-3".to_string(), + task_kind: CompactionTaskKind::Tantivy, index_uid: quickwit_proto::types::IndexUid::for_test("test-index", 0), source_id: "src".to_string(), split_ids: vec!["s4".to_string()], diff --git a/quickwit/quickwit-compaction/src/lib.rs b/quickwit/quickwit-compaction/src/lib.rs index 98e5fc9a3d1..11d93de2aa5 100644 --- a/quickwit/quickwit-compaction/src/lib.rs +++ b/quickwit/quickwit-compaction/src/lib.rs @@ -18,6 +18,9 @@ mod compaction_pipeline; #[allow(dead_code)] mod compactor_supervisor; +#[cfg(feature = "metrics")] +#[allow(dead_code)] +mod parquet; pub mod planner; use std::sync::Arc; diff --git a/quickwit/quickwit-compaction/src/parquet/mod.rs b/quickwit/quickwit-compaction/src/parquet/mod.rs new file mode 100644 index 00000000000..8c8da192073 --- /dev/null +++ b/quickwit/quickwit-compaction/src/parquet/mod.rs @@ -0,0 +1,20 @@ +// 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. + +mod pipeline; +mod planner; +mod state; + +pub(crate) use pipeline::ParquetCompactionPipeline; +pub(crate) use planner::ParquetCompactionPlanner; diff --git a/quickwit/quickwit-compaction/src/parquet/pipeline.rs b/quickwit/quickwit-compaction/src/parquet/pipeline.rs new file mode 100644 index 00000000000..34050b5c7f6 --- /dev/null +++ b/quickwit/quickwit-compaction/src/parquet/pipeline.rs @@ -0,0 +1,458 @@ +// 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. + +use std::path::Path; +use std::sync::Arc; + +use anyhow::Context; +use async_trait::async_trait; +use quickwit_actors::{ + Actor, ActorContext, ActorExitStatus, ActorHandle, Handler, Health, Mailbox, QueueCapacity, + SpawnContext, Supervisable, +}; +use quickwit_common::KillSwitch; +use quickwit_common::temp_dir::TempDirectory; +use quickwit_common::thread_pool::run_cpu_intensive; +use quickwit_indexing::actors::{ + ParquetSplitBatch, ParquetUploader, Publisher, Sequencer, UploaderType, +}; +use quickwit_indexing::models::PublishLock; +use quickwit_parquet_engine::merge::metadata_aggregation::merge_parquet_split_metadata; +use quickwit_parquet_engine::merge::policy::ParquetMergeOperation; +use quickwit_parquet_engine::merge::{MergeConfig, MergeOutputFile, merge_sorted_parquet_files}; +use quickwit_parquet_engine::storage::ParquetWriterConfig; +use quickwit_proto::compaction::CompactionTaskKind; +use quickwit_proto::metastore::MetastoreServiceClient; +use quickwit_proto::types::{IndexUid, SourceId, SplitId}; +use quickwit_storage::Storage; +use tracing::{debug, error, info}; + +use crate::compaction_pipeline::{PipelineStatus, PipelineStatusUpdate}; + +const PARQUET_COMPACTION_PUBLISHER_NAME: &str = "ParquetCompactionPublisher"; + +struct ParquetCompactionExecutor { + scratch_directory: TempDirectory, + storage: Arc, + uploader_mailbox: Mailbox, + writer_config: ParquetWriterConfig, +} + +impl ParquetCompactionExecutor { + fn new( + scratch_directory: TempDirectory, + storage: Arc, + uploader_mailbox: Mailbox, + writer_config: ParquetWriterConfig, + ) -> Self { + Self { + scratch_directory, + storage, + uploader_mailbox, + writer_config, + } + } +} + +#[async_trait] +impl Actor for ParquetCompactionExecutor { + type ObservableState = (); + + fn observable_state(&self) {} + + fn name(&self) -> String { + "ParquetCompactionExecutor".to_string() + } + + fn queue_capacity(&self) -> QueueCapacity { + QueueCapacity::Bounded(1) + } +} + +#[async_trait] +impl Handler for ParquetCompactionExecutor { + type Reply = (); + + async fn handle( + &mut self, + operation: ParquetMergeOperation, + ctx: &ActorContext, + ) -> Result<(), ActorExitStatus> { + let merge_split_id = operation.merge_split_id.to_string(); + let num_inputs = operation.splits.len(); + info!( + merge_split_id = %merge_split_id, + num_inputs, + total_bytes = operation.total_size_bytes(), + "executing parquet compaction task" + ); + + let download_dir = self + .scratch_directory + .named_temp_child("parquet-merge-download-") + .context("failed to create parquet merge download directory") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + let mut downloaded_paths = Vec::with_capacity(num_inputs); + for split in &operation.splits { + if ctx.kill_switch().is_dead() { + return Err(ActorExitStatus::Killed); + } + + let parquet_filename = split.parquet_filename(); + let local_path = download_dir.path().join(&parquet_filename); + let _protect_guard = ctx.protect_zone(); + self.storage + .copy_to_file(Path::new(&parquet_filename), &local_path) + .await + .with_context(|| { + format!( + "failed to download parquet split {} from {}", + split.split_id, parquet_filename + ) + }) + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + downloaded_paths.push(local_path); + ctx.record_progress(); + } + + let output_dir = self.scratch_directory.path().join("merged_output"); + std::fs::create_dir_all(&output_dir) + .context("failed to create parquet merge output directory") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + + let input_paths = downloaded_paths.clone(); + let output_dir_clone = output_dir.clone(); + let writer_config = self.writer_config.clone(); + let outputs: Vec = run_cpu_intensive(move || { + let merge_config = MergeConfig { + num_outputs: 1, + writer_config, + }; + merge_sorted_parquet_files(&input_paths, &output_dir_clone, &merge_config) + }) + .await + .context("parquet merge task panicked") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))? + .context("parquet merge task failed") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + + let input_splits = &operation.splits; + let index_uid: IndexUid = input_splits[0] + .index_uid + .parse() + .context("invalid index_uid in parquet merge input") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + let replaced_split_ids: Vec = input_splits + .iter() + .map(|split| split.split_id.as_str().to_string()) + .collect(); + + let mut merged_splits = Vec::with_capacity(outputs.len()); + for output in &outputs { + let mut metadata = merge_parquet_split_metadata(input_splits, output) + .context("failed to build parquet merge output metadata") + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + metadata.split_id = operation.merge_split_id.clone(); + metadata.parquet_file = format!("{}.parquet", metadata.split_id); + + let expected_path = output_dir.join(&metadata.parquet_file); + if output.path != expected_path { + std::fs::rename(&output.path, &expected_path) + .with_context(|| { + format!( + "failed to rename parquet merge output {} to {}", + output.path.display(), + expected_path.display() + ) + }) + .map_err(|error| ActorExitStatus::from(anyhow::anyhow!(error)))?; + } + + info!( + split_id = %metadata.split_id, + num_rows = metadata.num_rows, + size_bytes = metadata.size_bytes, + "parquet compaction produced output split" + ); + merged_splits.push(metadata); + } + + if merged_splits.is_empty() { + debug!( + merge_split_id = %merge_split_id, + num_replaced = replaced_split_ids.len(), + "parquet compaction produced no output splits" + ); + } + + let batch = ParquetSplitBatch { + index_uid, + splits: merged_splits, + output_dir, + checkpoint_delta_opt: None, + publish_lock: PublishLock::default(), + publish_token_opt: None, + replaced_split_ids, + _scratch_directory_opt: Some(self.scratch_directory.clone()), + _merge_task_opt: None, + }; + ctx.send_message(&self.uploader_mailbox, batch).await?; + drop(download_dir); + Ok(()) + } +} + +struct ParquetCompactionPipelineHandles { + executor: ActorHandle, + uploader: ActorHandle, + sequencer: ActorHandle>, + publisher: ActorHandle, +} + +pub(crate) struct ParquetCompactionPipeline { + task_id: String, + merge_operation: ParquetMergeOperation, + index_uid: IndexUid, + status: PipelineStatus, + kill_switch: KillSwitch, + scratch_directory: TempDirectory, + storage: Arc, + metastore: MetastoreServiceClient, + max_concurrent_split_uploads: usize, + writer_config: ParquetWriterConfig, + handles: Option, +} + +impl ParquetCompactionPipeline { + pub fn new( + task_id: String, + scratch_directory: TempDirectory, + merge_operation: ParquetMergeOperation, + index_uid: IndexUid, + storage: Arc, + metastore: MetastoreServiceClient, + max_concurrent_split_uploads: usize, + writer_config: ParquetWriterConfig, + ) -> Self { + Self { + task_id, + merge_operation, + index_uid, + status: PipelineStatus::InProgress, + kill_switch: KillSwitch::default(), + scratch_directory, + storage, + metastore, + max_concurrent_split_uploads, + writer_config, + handles: None, + } + } + + pub fn status(&self) -> &PipelineStatus { + &self.status + } + + pub fn pipeline_status_update(&mut self) -> PipelineStatusUpdate { + self.update_status(); + self.build_status_update() + } + + fn supervisables(&self) -> Vec<&dyn Supervisable> { + let Some(handles) = &self.handles else { + return Vec::new(); + }; + vec![ + &handles.executor, + &handles.uploader, + &handles.sequencer, + &handles.publisher, + ] + } + + fn update_status(&mut self) { + if matches!( + self.status, + PipelineStatus::Completed | PipelineStatus::Failed { .. } + ) { + return; + } + if self.handles.is_none() { + return; + } + + let mut has_healthy = false; + let mut failure_actor_names = Vec::new(); + for supervisable in self.supervisables() { + match supervisable.check_health(true) { + Health::Healthy => has_healthy = true, + Health::FailureOrUnhealthy => { + failure_actor_names.push(supervisable.name().to_string()); + } + Health::Success => {} + } + } + + if !failure_actor_names.is_empty() { + let error_msg = format!("failed actors: {:?}", failure_actor_names); + error!(task_id=%self.task_id, "{error_msg}"); + self.status = PipelineStatus::Failed { error: error_msg }; + return; + } + if !has_healthy { + debug!(task_id=%self.task_id, "all parquet compaction actors completed"); + self.status = PipelineStatus::Completed; + } + } + + fn build_status_update(&self) -> PipelineStatusUpdate { + PipelineStatusUpdate { + task_id: self.task_id.clone(), + task_kind: CompactionTaskKind::Parquet, + index_uid: self.index_uid.clone(), + source_id: SourceId::default(), + split_ids: self + .merge_operation + .splits_as_slice() + .iter() + .map(|split| split.split_id.as_str().to_string()) + .collect::>(), + merged_split_id: self.merge_operation.merge_split_id.to_string(), + status: self.status.clone(), + } + } + + pub fn spawn_pipeline(&mut self, spawn_ctx: &SpawnContext) -> anyhow::Result<()> { + info!( + task_id = %self.task_id, + index_uid = %self.index_uid, + "spawning parquet compaction pipeline" + ); + + let publisher = Publisher::new( + PARQUET_COMPACTION_PUBLISHER_NAME, + QueueCapacity::Unbounded, + self.metastore.clone(), + None, + None, + ); + let (publisher_mailbox, publisher_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) + .spawn(publisher); + + let sequencer = Sequencer::new(publisher_mailbox); + let (sequencer_mailbox, sequencer_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) + .spawn(sequencer); + + let uploader = ParquetUploader::new( + UploaderType::MergeUploader, + self.metastore.clone(), + self.storage.clone(), + sequencer_mailbox, + self.max_concurrent_split_uploads, + ); + let (uploader_mailbox, uploader_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) + .spawn(uploader); + + let executor = ParquetCompactionExecutor::new( + self.scratch_directory.clone(), + self.storage.clone(), + uploader_mailbox, + self.writer_config.clone(), + ); + let (executor_mailbox, executor_handle) = spawn_ctx + .spawn_builder() + .set_kill_switch(self.kill_switch.child()) + .spawn(executor); + + executor_mailbox + .try_send_message(self.merge_operation.clone()) + .map_err(|error| { + anyhow::anyhow!("failed to send parquet merge operation to executor: {error:?}") + })?; + + self.handles = Some(ParquetCompactionPipelineHandles { + executor: executor_handle, + uploader: uploader_handle, + sequencer: sequencer_handle, + publisher: publisher_handle, + }); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + use std::sync::Arc; + use std::time::SystemTime; + + use quickwit_parquet_engine::split::{ + ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TimeRange, + }; + use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; + use quickwit_storage::RamStorage; + + use super::*; + + fn test_split(split_id: &str) -> ParquetSplitMetadata { + ParquetSplitMetadata { + kind: ParquetSplitKind::Metrics, + split_id: ParquetSplitId::new(split_id), + index_uid: "datadog-metrics:00000000000000000000000000".to_string(), + partition_id: 0, + time_range: TimeRange::new(1000, 2000), + num_rows: 100, + size_bytes: 1_000_000, + metric_names: HashSet::new(), + low_cardinality_tags: Default::default(), + high_cardinality_tag_keys: Default::default(), + created_at: SystemTime::now(), + parquet_file: format!("{split_id}.parquet"), + window: Some(0..3600), + sort_fields: "metric_name|host|timestamp_secs/V2".to_string(), + num_merge_ops: 0, + row_keys_proto: None, + zonemap_regexes: Default::default(), + } + } + + #[test] + fn test_status_update_unspawned_pipeline() { + let index_uid = IndexUid::for_test("datadog-metrics", 0); + let operation = ParquetMergeOperation::new(vec![test_split("s1"), test_split("s2")]); + let mut pipeline = ParquetCompactionPipeline::new( + "task-1".to_string(), + TempDirectory::for_test(), + operation, + index_uid.clone(), + Arc::new(RamStorage::default()), + MetastoreServiceClient::from_mock(MockMetastoreService::new()), + 2, + ParquetWriterConfig::default(), + ); + + let update = pipeline.pipeline_status_update(); + + assert_eq!(update.task_kind, CompactionTaskKind::Parquet); + assert_eq!(update.index_uid, index_uid); + assert_eq!(update.split_ids, vec!["s1".to_string(), "s2".to_string()]); + assert_eq!(update.status, PipelineStatus::InProgress); + } +} diff --git a/quickwit/quickwit-compaction/src/parquet/planner.rs b/quickwit/quickwit-compaction/src/parquet/planner.rs new file mode 100644 index 00000000000..4cf905260e5 --- /dev/null +++ b/quickwit/quickwit-compaction/src/parquet/planner.rs @@ -0,0 +1,469 @@ +// 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. + +use std::collections::HashMap; +use std::str::FromStr; +use std::sync::Arc; +use std::time::{Duration, SystemTime}; + +use anyhow::Result; +use quickwit_common::{is_parquet_pipeline_index, is_sketches_index}; +use quickwit_indexing::merge_policy::parquet_merge_policy_from_settings; +use quickwit_metastore::{ + IndexMetadata, ListIndexesMetadataResponseExt, ListParquetSplitsQuery, + ListParquetSplitsRequestExt, ListParquetSplitsResponseExt, ParquetSplitRecord, +}; +use quickwit_parquet_engine::merge::policy::{ + ParquetMergeOperation, ParquetMergePolicy, ParquetSplitMaturity, +}; +use quickwit_parquet_engine::split::ParquetSplitMetadata; +use quickwit_proto::compaction::{ + CompactionFailure, CompactionInProgress, CompactionSuccess, CompactionTaskKind, + MergeTaskAssignment, +}; +use quickwit_proto::metastore::{ + ListIndexesMetadataRequest, ListMetricsSplitsRequest, ListSketchSplitsRequest, + MetastoreService, MetastoreServiceClient, +}; +use quickwit_proto::types::{IndexUid, NodeId}; +use time::OffsetDateTime; +use tracing::{error, info}; +use ulid::Ulid; + +use super::state::ParquetCompactionState; + +#[derive(Clone)] +struct ParquetIndexEntry { + index_metadata: IndexMetadata, + merge_policy: Arc, +} + +impl ParquetIndexEntry { + fn new(index_metadata: IndexMetadata) -> Self { + let merge_policy = + parquet_merge_policy_from_settings(&index_metadata.index_config.indexing_settings); + Self { + index_metadata, + merge_policy, + } + } + + fn is_split_mature(&self, split: &ParquetSplitMetadata) -> bool { + match self + .merge_policy + .split_maturity(split.size_bytes, split.num_merge_ops) + { + ParquetSplitMaturity::Mature => true, + ParquetSplitMaturity::Immature { + maturation_period, .. + } => split.created_at + maturation_period <= SystemTime::now(), + } + } + + fn doc_mapping_json(&self) -> String { + serde_json::to_string(&self.index_metadata.index_config.doc_mapping) + .expect("doc mapping serialization should not fail") + } + + fn search_settings_json(&self) -> String { + serde_json::to_string(&self.index_metadata.index_config.search_settings) + .expect("search settings serialization should not fail") + } + + fn indexing_settings_json(&self) -> String { + serde_json::to_string(&self.index_metadata.index_config.indexing_settings) + .expect("indexing settings serialization should not fail") + } + + fn retention_policy_json(&self) -> String { + match &self.index_metadata.index_config.retention_policy_opt { + Some(policy) => serde_json::to_string(policy) + .expect("retention policy serialization should not fail"), + None => String::new(), + } + } + + fn index_storage_uri(&self) -> String { + self.index_metadata.index_config.index_uri.to_string() + } +} + +pub(crate) struct ParquetCompactionPlanner { + state: ParquetCompactionState, + indexes: HashMap, + cursor: i64, + metastore: MetastoreServiceClient, +} + +const STARTUP_LOOKBACK: Duration = Duration::from_secs(24 * 60 * 60); + +impl ParquetCompactionPlanner { + pub fn new(metastore: MetastoreServiceClient) -> Self { + let cursor = OffsetDateTime::now_utc().unix_timestamp() - STARTUP_LOOKBACK.as_secs() as i64; + Self { + state: ParquetCompactionState::new(), + indexes: HashMap::new(), + cursor, + metastore, + } + } + + pub async fn scan_and_plan(&mut self) -> Result<()> { + self.refresh_indexes().await?; + let splits = self.scan_metastore().await?; + self.ingest_splits(splits); + self.run_merge_policies(); + Ok(()) + } + + pub fn process_successes(&mut self, successes: &[CompactionSuccess]) { + self.state.process_successes(successes); + } + + pub fn process_failures(&mut self, failures: &[CompactionFailure]) { + self.state.process_failures(failures); + } + + pub fn update_heartbeats(&mut self, node_id: &NodeId, in_progress: &[CompactionInProgress]) { + self.state.update_heartbeats(node_id, in_progress); + } + + pub fn check_heartbeat_timeouts(&mut self) { + self.state.check_heartbeat_timeouts(); + } + + pub fn assign_tasks( + &mut self, + node_id: &NodeId, + available_slots: u32, + ) -> Vec { + let pending = self.state.pop_pending(available_slots as usize); + let mut assignments = Vec::with_capacity(pending.len()); + + for (scope, operation) in pending { + let Ok(index_uid) = IndexUid::from_str(&scope.index_uid) else { + error!(index_uid=%scope.index_uid, "invalid parquet index uid in pending operation, skipping"); + continue; + }; + let Some(index_entry) = self.indexes.get(&index_uid) else { + error!(index_uid=%index_uid, "parquet index config not found for pending operation, skipping"); + continue; + }; + + let task_id = Ulid::new().to_string(); + let assignment = build_task_assignment(&task_id, index_entry, &operation, &index_uid); + let split_ids = operation + .splits_as_slice() + .iter() + .map(|split| split.split_id.as_str().to_string()) + .collect(); + self.state + .record_assignment(task_id, split_ids, node_id.clone()); + assignments.push(assignment); + } + assignments + } + + async fn refresh_indexes(&mut self) -> Result<()> { + let index_metadata_list = self + .metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) + .await? + .deserialize_indexes_metadata() + .await?; + + self.indexes.clear(); + for index_metadata in index_metadata_list { + if !is_parquet_pipeline_index(&index_metadata.index_uid.index_id) { + continue; + } + self.indexes.insert( + index_metadata.index_uid.clone(), + ParquetIndexEntry::new(index_metadata), + ); + } + Ok(()) + } + + async fn scan_metastore(&self) -> Result> { + let mut splits = Vec::new(); + for index_uid in self.indexes.keys() { + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) + .retain_immature(OffsetDateTime::now_utc()) + .with_update_timestamp_gte(self.cursor); + let mut index_splits = self.list_index_splits(index_uid, &query).await?; + splits.append(&mut index_splits); + } + if !splits.is_empty() { + info!( + num_splits = splits.len(), + "fetched published parquet splits for compaction planning" + ); + } + Ok(splits) + } + + async fn list_index_splits( + &self, + index_uid: &IndexUid, + query: &ListParquetSplitsQuery, + ) -> Result> { + let records = if is_sketches_index(&index_uid.index_id) { + let request = ListSketchSplitsRequest::try_from_query(index_uid.clone(), query)?; + self.metastore + .list_sketch_splits(request) + .await? + .deserialize_splits()? + } else { + let request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), query)?; + self.metastore + .list_metrics_splits(request) + .await? + .deserialize_splits()? + }; + Ok(records) + } + + fn ingest_splits(&mut self, splits: Vec) { + for split_record in splits { + let split = split_record.metadata; + if self.state.is_split_known(split.split_id.as_str()) { + continue; + } + let Ok(index_uid) = IndexUid::from_str(&split.index_uid) else { + error!(index_uid=%split.index_uid, split_id=%split.split_id, "invalid index uid on parquet split, skipping split"); + continue; + }; + let Some(index_entry) = self.indexes.get(&index_uid) else { + error!(index_uid=%index_uid, split_id=%split.split_id, "parquet index config not found, skipping split"); + continue; + }; + if index_entry.is_split_mature(&split) { + continue; + } + self.cursor = self.cursor.max(split_record.update_timestamp); + self.state.track_split(split); + } + } + + fn run_merge_policies(&mut self) { + for scope in self.state.scope_keys() { + let Ok(index_uid) = IndexUid::from_str(&scope.index_uid) else { + error!(index_uid=%scope.index_uid, "invalid parquet index uid in compaction scope, skipping"); + continue; + }; + if let Some(index_entry) = self.indexes.get(&index_uid) { + self.state.plan_scope(&scope, &index_entry.merge_policy); + } + } + } +} + +fn build_task_assignment( + task_id: &str, + index_entry: &ParquetIndexEntry, + operation: &ParquetMergeOperation, + index_uid: &IndexUid, +) -> MergeTaskAssignment { + MergeTaskAssignment { + task_id: task_id.to_string(), + splits_metadata_json: operation + .splits_as_slice() + .iter() + .map(|split| { + serde_json::to_string(split) + .expect("parquet split metadata serialization should not fail") + }) + .collect(), + doc_mapping_json: index_entry.doc_mapping_json(), + search_settings_json: index_entry.search_settings_json(), + indexing_settings_json: index_entry.indexing_settings_json(), + retention_policy_json: index_entry.retention_policy_json(), + index_uid: Some(index_uid.clone()), + source_id: String::new(), + index_storage_uri: index_entry.index_storage_uri(), + task_kind: CompactionTaskKind::Parquet as i32, + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + use std::ops::Bound; + use std::time::SystemTime; + + use quickwit_metastore::{IndexMetadata, SplitState}; + use quickwit_parquet_engine::split::{ + ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TimeRange, + }; + use quickwit_proto::metastore::{ + ListMetricsSplitsResponse, MetastoreServiceClient, MockMetastoreService, + }; + use quickwit_proto::types::{IndexUid, NodeId}; + + use super::*; + + fn test_index_entry() -> ParquetIndexEntry { + let mut index_metadata = + IndexMetadata::for_test("datadog-metrics", "ram:///datadog-metrics"); + index_metadata + .index_config + .indexing_settings + .parquet_merge_policy = Some( + quickwit_config::merge_policy_config::ParquetMergePolicyConfig { + merge_factor: 2, + max_merge_factor: 2, + maturation_period: std::time::Duration::from_secs(3600), + ..Default::default() + }, + ); + ParquetIndexEntry::new(index_metadata) + } + + fn test_split(split_id: &str, index_uid: &IndexUid) -> ParquetSplitMetadata { + ParquetSplitMetadata { + kind: ParquetSplitKind::Metrics, + split_id: ParquetSplitId::new(split_id), + index_uid: index_uid.to_string(), + partition_id: 0, + time_range: TimeRange::new(1000, 2000), + num_rows: 100, + size_bytes: 1_000_000, + metric_names: HashSet::new(), + low_cardinality_tags: Default::default(), + high_cardinality_tag_keys: Default::default(), + created_at: SystemTime::now(), + parquet_file: format!("{split_id}.parquet"), + window: Some(0..3600), + sort_fields: "metric_name|host|timestamp_secs/V2".to_string(), + num_merge_ops: 0, + row_keys_proto: None, + zonemap_regexes: Default::default(), + } + } + + fn test_split_record( + split_id: &str, + index_uid: &IndexUid, + update_timestamp: i64, + ) -> ParquetSplitRecord { + ParquetSplitRecord { + state: SplitState::Published, + update_timestamp, + metadata: test_split(split_id, index_uid), + } + } + + #[test] + fn test_new_starts_cursor_with_24h_lookback() { + let before = OffsetDateTime::now_utc().unix_timestamp(); + let planner = ParquetCompactionPlanner::new(MetastoreServiceClient::mocked()); + let after = OffsetDateTime::now_utc().unix_timestamp(); + + assert!(planner.cursor >= before - STARTUP_LOOKBACK.as_secs() as i64); + assert!(planner.cursor <= after - STARTUP_LOOKBACK.as_secs() as i64); + } + + #[tokio::test] + async fn test_scan_metastore_uses_cursor() { + let expected_cursor = 12_345; + + let mut mock = MockMetastoreService::new(); + mock.expect_list_metrics_splits().returning(move |request| { + let query = request.deserialize_query().unwrap(); + assert_eq!( + query.update_timestamp.start, + Bound::Included(expected_cursor) + ); + assert!(matches!(query.mature, Bound::Excluded(_))); + assert_eq!(query.split_states, vec![SplitState::Published]); + Ok(ListMetricsSplitsResponse::try_from_splits(&[]).unwrap()) + }); + + let mut planner = ParquetCompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); + planner.cursor = expected_cursor; + let index_entry = test_index_entry(); + let index_uid = index_entry.index_metadata.index_uid.clone(); + planner.indexes.insert(index_uid, index_entry); + + let splits = planner.scan_metastore().await.unwrap(); + assert!(splits.is_empty()); + } + + #[test] + fn test_ingest_splits_advances_cursor_for_tracked_splits() { + let index_entry = test_index_entry(); + let index_uid = index_entry.index_metadata.index_uid.clone(); + + let mut planner = ParquetCompactionPlanner::new(MetastoreServiceClient::mocked()); + planner.cursor = 0; + planner.indexes.insert(index_uid.clone(), index_entry); + + planner.state.track_split(test_split("known", &index_uid)); + planner.ingest_splits(vec![ + test_split_record("known", &index_uid, 1000), + test_split_record("fresh", &index_uid, 3000), + ]); + + assert!(planner.state.is_split_known("known")); + assert!(planner.state.is_split_known("fresh")); + assert_eq!(planner.cursor, 3000); + } + + #[test] + fn test_build_task_assignment_marks_parquet_kind() { + let index_entry = test_index_entry(); + let index_uid = index_entry.index_metadata.index_uid.clone(); + let operation = ParquetMergeOperation::new(vec![ + test_split("s1", &index_uid), + test_split("s2", &index_uid), + ]); + + let assignment = build_task_assignment("task-1", &index_entry, &operation, &index_uid); + + assert_eq!(assignment.task_kind, CompactionTaskKind::Parquet as i32); + assert_eq!(assignment.splits_metadata_json.len(), 2); + assert_eq!(assignment.index_uid, Some(index_uid)); + assert_eq!(assignment.source_id, String::new()); + assert_eq!(assignment.doc_mapping_json, index_entry.doc_mapping_json()); + assert_eq!( + assignment.search_settings_json, + index_entry.search_settings_json() + ); + assert_eq!( + assignment.retention_policy_json, + index_entry.retention_policy_json() + ); + assert!(!assignment.indexing_settings_json.is_empty()); + } + + #[test] + fn test_assign_tasks_sets_parquet_kind() { + let index_entry = test_index_entry(); + let index_uid = index_entry.index_metadata.index_uid.clone(); + + let mut planner = ParquetCompactionPlanner::new(MetastoreServiceClient::mocked()); + planner.indexes.insert(index_uid.clone(), index_entry); + planner.ingest_splits(vec![ + test_split_record("s1", &index_uid, 1000), + test_split_record("s2", &index_uid, 2000), + ]); + planner.run_merge_policies(); + + let assignments = planner.assign_tasks(&NodeId::from("test-node"), 1); + assert_eq!(assignments.len(), 1); + assert_eq!(assignments[0].task_kind, CompactionTaskKind::Parquet as i32); + assert_eq!(assignments[0].splits_metadata_json.len(), 2); + } +} diff --git a/quickwit/quickwit-compaction/src/parquet/state.rs b/quickwit/quickwit-compaction/src/parquet/state.rs new file mode 100644 index 00000000000..f2694cca727 --- /dev/null +++ b/quickwit/quickwit-compaction/src/parquet/state.rs @@ -0,0 +1,277 @@ +// 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. + +use std::collections::{HashMap, HashSet, VecDeque}; +use std::sync::Arc; +use std::time::{Duration, Instant}; + +use quickwit_parquet_engine::merge::policy::{ + CompactionScope, ParquetMergeOperation, ParquetMergePolicy, +}; +use quickwit_parquet_engine::split::ParquetSplitMetadata; +use quickwit_proto::compaction::{CompactionFailure, CompactionInProgress, CompactionSuccess}; +use quickwit_proto::types::{NodeId, SplitId}; +use tracing::{error, info, warn}; + +const HEARTBEAT_TIMEOUT: Duration = Duration::from_secs(60); + +struct InFlightParquetCompaction { + split_ids: Vec, + node_id: NodeId, + last_heartbeat: Instant, +} + +/// Tracks Parquet split-level state for the compaction planner. +/// +/// This intentionally mirrors the Tantivy `CompactionState`, but keys splits +/// by Parquet `CompactionScope` so sort schema, partition, and time-window +/// invariants are preserved before merge policy execution. +pub struct ParquetCompactionState { + needs_compaction: HashMap>, + needs_compaction_split_ids: HashSet, + in_flight: HashMap, + in_flight_split_ids: HashSet, + pending_operations: VecDeque<(CompactionScope, ParquetMergeOperation)>, +} + +impl ParquetCompactionState { + pub fn new() -> Self { + ParquetCompactionState { + needs_compaction: HashMap::new(), + needs_compaction_split_ids: HashSet::new(), + in_flight: HashMap::new(), + in_flight_split_ids: HashSet::new(), + pending_operations: VecDeque::new(), + } + } + + pub fn is_split_known(&self, split_id: &str) -> bool { + self.needs_compaction_split_ids.contains(split_id) + || self.in_flight_split_ids.contains(split_id) + } + + pub fn track_split(&mut self, split: ParquetSplitMetadata) { + let Some(scope) = CompactionScope::from_split(&split) else { + return; + }; + let split_id = split.split_id.as_str().to_string(); + self.needs_compaction_split_ids.insert(split_id); + self.needs_compaction.entry(scope).or_default().push(split); + } + + pub fn scope_keys(&self) -> Vec { + self.needs_compaction.keys().cloned().collect() + } + + pub fn plan_scope( + &mut self, + scope: &CompactionScope, + merge_policy: &Arc, + ) { + let Some(splits) = self.needs_compaction.get_mut(scope) else { + return; + }; + for operation in merge_policy.operations(splits) { + for split in operation.splits_as_slice() { + let split_id = split.split_id.as_str(); + self.needs_compaction_split_ids.remove(split_id); + self.in_flight_split_ids.insert(split_id.to_string()); + } + self.pending_operations + .push_back((scope.clone(), operation)); + } + if splits.is_empty() { + self.needs_compaction.remove(scope); + } + } + + pub fn process_successes(&mut self, successes: &[CompactionSuccess]) { + for success in successes { + if let Some(inflight) = self.in_flight.remove(&success.task_id) { + info!(task_id=%success.task_id, "parquet compaction task completed"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + pub fn process_failures(&mut self, failures: &[CompactionFailure]) { + for failure in failures { + if let Some(inflight) = self.in_flight.remove(&failure.task_id) { + warn!(task_id=%failure.task_id, error=%failure.error_message, "parquet compaction task failed"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + pub fn update_heartbeats(&mut self, node_id: &NodeId, in_progress: &[CompactionInProgress]) { + for task in in_progress { + if let Some(inflight) = self.in_flight.get_mut(&task.task_id) { + inflight.last_heartbeat = Instant::now(); + } else { + for split_id in &task.split_ids { + self.in_flight_split_ids.insert(split_id.clone()); + self.needs_compaction_split_ids.remove(split_id.as_str()); + } + self.in_flight.insert( + task.task_id.clone(), + InFlightParquetCompaction { + split_ids: task.split_ids.clone(), + node_id: node_id.clone(), + last_heartbeat: Instant::now(), + }, + ); + } + } + } + + pub fn check_heartbeat_timeouts(&mut self) { + let now = Instant::now(); + let timed_out_task_ids: Vec = self + .in_flight + .iter() + .filter(|(_, inflight)| now.duration_since(inflight.last_heartbeat) > HEARTBEAT_TIMEOUT) + .map(|(task_id, _)| task_id.clone()) + .collect(); + + for task_id in timed_out_task_ids { + if let Some(inflight) = self.in_flight.remove(&task_id) { + error!(task_id=%task_id, node_id=%inflight.node_id, "parquet compaction task timed out"); + for split_id in &inflight.split_ids { + self.in_flight_split_ids.remove(split_id.as_str()); + } + } + } + } + + pub fn pop_pending(&mut self, count: usize) -> Vec<(CompactionScope, ParquetMergeOperation)> { + let count = count.min(self.pending_operations.len()); + let mut operations = Vec::with_capacity(count); + for _ in 0..count { + operations.push(self.pending_operations.pop_front().unwrap()); + } + operations + } + + pub fn record_assignment(&mut self, task_id: String, split_ids: Vec, node_id: NodeId) { + self.in_flight.insert( + task_id, + InFlightParquetCompaction { + split_ids, + node_id, + last_heartbeat: Instant::now(), + }, + ); + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashSet; + use std::time::{Duration, SystemTime}; + + use quickwit_parquet_engine::merge::policy::{ + ConstWriteAmplificationParquetMergePolicy, ParquetMergePolicyConfig, + }; + use quickwit_parquet_engine::split::{ + ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TimeRange, + }; + use quickwit_proto::compaction::CompactionTaskKind; + + use super::*; + + fn test_merge_policy() -> Arc { + Arc::new(ConstWriteAmplificationParquetMergePolicy::new( + ParquetMergePolicyConfig { + merge_factor: 2, + max_merge_factor: 2, + target_split_size_bytes: 256 * 1024 * 1024, + maturation_period: Duration::from_secs(3600), + ..Default::default() + }, + )) + } + + fn test_split(split_id: &str, window_start: Option) -> ParquetSplitMetadata { + let mut split = ParquetSplitMetadata { + kind: ParquetSplitKind::Metrics, + split_id: ParquetSplitId::new(split_id), + index_uid: "test-index:00000000000000000000000000".to_string(), + partition_id: 0, + time_range: TimeRange::new(1000, 2000), + num_rows: 100, + size_bytes: 1_000_000, + metric_names: HashSet::new(), + low_cardinality_tags: Default::default(), + high_cardinality_tag_keys: Default::default(), + created_at: SystemTime::now(), + parquet_file: format!("{split_id}.parquet"), + window: None, + sort_fields: "metric_name|host|timestamp_secs/V2".to_string(), + num_merge_ops: 0, + row_keys_proto: None, + zonemap_regexes: Default::default(), + }; + split.window = window_start.map(|start| start..start + 3600); + split + } + + #[test] + fn test_track_split_skips_splits_without_scope() { + let mut state = ParquetCompactionState::new(); + + state.track_split(test_split("s1", None)); + + assert!(!state.is_split_known("s1")); + assert!(state.scope_keys().is_empty()); + } + + #[test] + fn test_plan_scope_moves_splits_to_in_flight() { + let mut state = ParquetCompactionState::new(); + let merge_policy = test_merge_policy(); + + state.track_split(test_split("s1", Some(0))); + state.track_split(test_split("s2", Some(0))); + + let scopes = state.scope_keys(); + assert_eq!(scopes.len(), 1); + state.plan_scope(&scopes[0], &merge_policy); + + assert_eq!(state.pending_operations.len(), 1); + assert!(!state.needs_compaction_split_ids.contains("s1")); + assert!(!state.needs_compaction_split_ids.contains("s2")); + assert!(state.in_flight_split_ids.contains("s1")); + assert!(state.in_flight_split_ids.contains("s2")); + } + + #[test] + fn test_process_success_clears_in_flight_ids() { + let node_id = NodeId::from("worker-1"); + let mut state = ParquetCompactionState::new(); + state.in_flight_split_ids.insert("s1".to_string()); + state.record_assignment("task-1".to_string(), vec!["s1".to_string()], node_id); + + state.process_successes(&[CompactionSuccess { + task_id: "task-1".to_string(), + merged_split_id: "merged".to_string(), + task_kind: CompactionTaskKind::Parquet as i32, + }]); + + assert!(!state.is_split_known("s1")); + } +} diff --git a/quickwit/quickwit-compaction/src/planner/compaction_planner.rs b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs index 0fb6ccc6d50..745df4b9b2d 100644 --- a/quickwit/quickwit-compaction/src/planner/compaction_planner.rs +++ b/quickwit/quickwit-compaction/src/planner/compaction_planner.rs @@ -23,7 +23,8 @@ use quickwit_metastore::{ ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitState, }; use quickwit_proto::compaction::{ - CompactionResult, MergeTaskAssignment, ReportStatusRequest, ReportStatusResponse, + CompactionResult, CompactionTaskKind, MergeTaskAssignment, ReportStatusRequest, + ReportStatusResponse, }; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::types::{IndexUid, NodeId, SourceId}; @@ -37,6 +38,8 @@ use super::index_config_store::{IndexConfigStore, IndexEntry}; pub struct CompactionPlanner { state: CompactionState, index_config_store: IndexConfigStore, + #[cfg(feature = "metrics")] + parquet_planner: crate::parquet::ParquetCompactionPlanner, cursor: i64, metastore: MetastoreServiceClient, } @@ -87,6 +90,8 @@ impl Handler for CompactionPlanner { error!(error=%err, "error scanning metastore and planning merges"); } self.state.check_heartbeat_timeouts(); + #[cfg(feature = "metrics")] + self.parquet_planner.check_heartbeat_timeouts(); ctx.schedule_self_msg(SCAN_AND_PLAN_INTERVAL, ScanAndPlan); Ok(()) } @@ -102,11 +107,47 @@ impl Handler for CompactionPlanner { _ctx: &ActorContext, ) -> Result, ActorExitStatus> { let node_id = NodeId::from(msg.node_id); - self.state.process_successes(&msg.successes); - self.state.process_failures(&msg.failures); - self.state.update_heartbeats(&node_id, &msg.in_progress); - let new_tasks = self.assign_tasks(&node_id, msg.available_slots); - Ok(Ok(ReportStatusResponse { new_tasks })) + #[cfg(feature = "metrics")] + { + let (parquet_successes, tantivy_successes): (Vec<_>, Vec<_>) = msg + .successes + .iter() + .cloned() + .partition(|success| is_parquet_task_kind(success.task_kind)); + let (parquet_failures, tantivy_failures): (Vec<_>, Vec<_>) = msg + .failures + .iter() + .cloned() + .partition(|failure| is_parquet_task_kind(failure.task_kind)); + let (parquet_in_progress, tantivy_in_progress): (Vec<_>, Vec<_>) = msg + .in_progress + .iter() + .cloned() + .partition(|task| is_parquet_task_kind(task.task_kind)); + + self.state.process_successes(&tantivy_successes); + self.state.process_failures(&tantivy_failures); + self.state.update_heartbeats(&node_id, &tantivy_in_progress); + + self.parquet_planner.process_successes(&parquet_successes); + self.parquet_planner.process_failures(&parquet_failures); + self.parquet_planner + .update_heartbeats(&node_id, &parquet_in_progress); + + let mut new_tasks = self.assign_tasks(&node_id, msg.available_slots); + let remaining_slots = msg.available_slots.saturating_sub(new_tasks.len() as u32); + new_tasks.extend(self.parquet_planner.assign_tasks(&node_id, remaining_slots)); + return Ok(Ok(ReportStatusResponse { new_tasks })); + } + + #[cfg(not(feature = "metrics"))] + { + self.state.process_successes(&msg.successes); + self.state.process_failures(&msg.failures); + self.state.update_heartbeats(&node_id, &msg.in_progress); + let new_tasks = self.assign_tasks(&node_id, msg.available_slots); + Ok(Ok(ReportStatusResponse { new_tasks })) + } } } @@ -118,6 +159,8 @@ impl CompactionPlanner { CompactionPlanner { state: CompactionState::new(), index_config_store: IndexConfigStore::new(metastore.clone()), + #[cfg(feature = "metrics")] + parquet_planner: crate::parquet::ParquetCompactionPlanner::new(metastore.clone()), cursor, metastore, } @@ -163,6 +206,8 @@ impl CompactionPlanner { let splits = self.scan_metastore().await?; self.ingest_splits(splits).await; self.run_merge_policies(); + #[cfg(feature = "metrics")] + self.parquet_planner.scan_and_plan().await?; Ok(()) } @@ -207,6 +252,14 @@ impl CompactionPlanner { } } +#[cfg(feature = "metrics")] +fn is_parquet_task_kind(task_kind: i32) -> bool { + matches!( + CompactionTaskKind::try_from(task_kind), + Ok(CompactionTaskKind::Parquet) + ) +} + fn build_task_assignment( task_id: &str, index_entry: &IndexEntry, @@ -230,6 +283,7 @@ fn build_task_assignment( index_uid: Some(index_uid.clone()), source_id: source_id.to_string(), index_storage_uri: index_entry.index_storage_uri(), + task_kind: CompactionTaskKind::Tantivy as i32, } } @@ -242,11 +296,15 @@ mod tests { use quickwit_config::merge_policy_config::{ ConstWriteAmplificationMergePolicyConfig, MergePolicyConfig, }; + #[cfg(feature = "metrics")] + use quickwit_metastore::ListIndexesMetadataResponseExt; use quickwit_metastore::{ IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt, Split, SplitMaturity, SplitMetadata, SplitState, }; use quickwit_proto::compaction::CompactionSuccess; + #[cfg(feature = "metrics")] + use quickwit_proto::metastore::ListIndexesMetadataResponse; use quickwit_proto::metastore::{ IndexMetadataResponse, ListSplitsResponse, MetastoreError, MockMetastoreService, }; @@ -415,6 +473,9 @@ mod tests { }); mock.expect_index_metadata() .returning(move |_| Ok(index_metadata_response.clone())); + #[cfg(feature = "metrics")] + mock.expect_list_indexes_metadata() + .returning(|_| Ok(ListIndexesMetadataResponse::for_test(Vec::new()))); let mut planner = CompactionPlanner::new(MetastoreServiceClient::from_mock(mock)); planner.cursor = 0; @@ -504,6 +565,7 @@ mod tests { planner.state.process_successes(&[CompactionSuccess { task_id, merged_split_id: "merged-1".to_string(), + task_kind: CompactionTaskKind::Tantivy as i32, }]); // The original splits are no longer tracked. Re-ingesting them diff --git a/quickwit/quickwit-compaction/src/planner/compaction_state.rs b/quickwit/quickwit-compaction/src/planner/compaction_state.rs index 1d1ea07e91d..34058ccff10 100644 --- a/quickwit/quickwit-compaction/src/planner/compaction_state.rs +++ b/quickwit/quickwit-compaction/src/planner/compaction_state.rs @@ -215,6 +215,7 @@ mod tests { ConstWriteAmplificationMergePolicyConfig, MergePolicyConfig, }; use quickwit_indexing::merge_policy::merge_policy_from_settings; + use quickwit_proto::compaction::CompactionTaskKind; use quickwit_proto::types::IndexUid; use super::*; @@ -319,6 +320,7 @@ mod tests { state.process_successes(&[CompactionSuccess { task_id: "task-1".to_string(), merged_split_id: "merged-1".to_string(), + task_kind: CompactionTaskKind::Tantivy as i32, }]); assert!(!state.is_split_known("s1")); assert!(!state.is_split_known("s2")); @@ -326,6 +328,7 @@ mod tests { state.process_failures(&[CompactionFailure { task_id: "task-2".to_string(), error_message: "boom".to_string(), + task_kind: CompactionTaskKind::Tantivy as i32, }]); assert!(!state.is_split_known("s3")); } @@ -348,6 +351,7 @@ mod tests { index_uid: Some(index_uid), source_id: "test-source".to_string(), split_ids: vec!["s1".to_string()], + task_kind: CompactionTaskKind::Tantivy as i32, }], ); diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 8752d348e02..59676362dd4 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -159,6 +159,18 @@ impl ListParquetSplitsQuery { self } + /// Filter splits updated at or after the given timestamp (epoch seconds). + pub fn with_update_timestamp_gte(mut self, timestamp: i64) -> Self { + self.update_timestamp.start = Bound::Included(timestamp); + self + } + + /// Retains splits that are immature at the given datetime. + pub fn retain_immature(mut self, now: OffsetDateTime) -> Self { + self.mature = Bound::Excluded(now); + self + } + /// Limit number of results returned. pub fn with_limit(mut self, limit: usize) -> Self { self.limit = Some(limit); diff --git a/quickwit/quickwit-proto/protos/quickwit/compaction.proto b/quickwit/quickwit-proto/protos/quickwit/compaction.proto index 7313d85db91..7326f32d205 100644 --- a/quickwit/quickwit-proto/protos/quickwit/compaction.proto +++ b/quickwit/quickwit-proto/protos/quickwit/compaction.proto @@ -22,6 +22,11 @@ service CompactionPlannerService { rpc ReportStatus(ReportStatusRequest) returns (ReportStatusResponse); } +enum CompactionTaskKind { + COMPACTION_TASK_KIND_TANTIVY = 0; + COMPACTION_TASK_KIND_PARQUET = 1; +} + message ReportStatusRequest { string node_id = 1; uint32 available_slots = 2; @@ -35,16 +40,19 @@ message CompactionInProgress { quickwit.common.IndexUid index_uid = 2; string source_id = 3; repeated string split_ids = 4; + CompactionTaskKind task_kind = 5; } message CompactionSuccess { string task_id = 1; string merged_split_id = 2; + CompactionTaskKind task_kind = 3; } message CompactionFailure { string task_id = 1; string error_message = 2; + CompactionTaskKind task_kind = 3; } message ReportStatusResponse { @@ -61,4 +69,5 @@ message MergeTaskAssignment { quickwit.common.IndexUid index_uid = 7; string source_id = 8; string index_storage_uri = 9; -} \ No newline at end of file + CompactionTaskKind task_kind = 10; +} diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs index f96bb33ed20..c5724449f41 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.compaction.rs @@ -24,6 +24,8 @@ pub struct CompactionInProgress { pub source_id: ::prost::alloc::string::String, #[prost(string, repeated, tag = "4")] pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(enumeration = "CompactionTaskKind", tag = "5")] + pub task_kind: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] @@ -32,6 +34,8 @@ pub struct CompactionSuccess { pub task_id: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub merged_split_id: ::prost::alloc::string::String, + #[prost(enumeration = "CompactionTaskKind", tag = "3")] + pub task_kind: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] @@ -40,6 +44,8 @@ pub struct CompactionFailure { pub task_id: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub error_message: ::prost::alloc::string::String, + #[prost(enumeration = "CompactionTaskKind", tag = "3")] + pub task_kind: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -68,6 +74,36 @@ pub struct MergeTaskAssignment { pub source_id: ::prost::alloc::string::String, #[prost(string, tag = "9")] pub index_storage_uri: ::prost::alloc::string::String, + #[prost(enumeration = "CompactionTaskKind", tag = "10")] + pub task_kind: i32, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum CompactionTaskKind { + Tantivy = 0, + Parquet = 1, +} +impl CompactionTaskKind { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + Self::Tantivy => "COMPACTION_TASK_KIND_TANTIVY", + Self::Parquet => "COMPACTION_TASK_KIND_PARQUET", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "COMPACTION_TASK_KIND_TANTIVY" => Some(Self::Tantivy), + "COMPACTION_TASK_KIND_PARQUET" => Some(Self::Parquet), + _ => None, + } + } } /// BEGIN quickwit-codegen #[allow(unused_imports)] diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index a6910cdb6a3..5bb83ba3651 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -113,6 +113,10 @@ quickwit-storage = { workspace = true, features = ["testsuite"] } [features] datafusion = ["dep:quickwit-datafusion"] +metrics = [ + "quickwit-compaction/metrics", + "quickwit-indexing/metrics", +] pprof = [ "dep:pprof" ] From 3608b71962d22c87fb050be3864f98b2fdceec7c Mon Sep 17 00:00:00 2001 From: Matthew Kim Date: Tue, 12 May 2026 15:44:21 -0400 Subject: [PATCH 11/11] some fixes --- .../src/compactor_supervisor.rs | 2 ++ .../src/parquet/pipeline.rs | 14 ++++++++- .../src/parquet/planner.rs | 31 +++++++++---------- .../quickwit-compaction/src/parquet/state.rs | 3 +- 4 files changed, 31 insertions(+), 19 deletions(-) diff --git a/quickwit/quickwit-compaction/src/compactor_supervisor.rs b/quickwit/quickwit-compaction/src/compactor_supervisor.rs index 508a361f15d..0d3485a7003 100644 --- a/quickwit/quickwit-compaction/src/compactor_supervisor.rs +++ b/quickwit/quickwit-compaction/src/compactor_supervisor.rs @@ -24,6 +24,8 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_common::uri::Uri; use quickwit_config::{IndexingSettings, RetentionPolicy, SearchSettings, build_doc_mapper}; use quickwit_doc_mapper::DocMapping; +#[cfg(feature = "metrics")] +use quickwit_indexing::merge_policy::parquet_merge_policy_from_settings; use quickwit_indexing::merge_policy::{MergeOperation, merge_policy_from_settings}; use quickwit_indexing::{IndexingSplitCache, IndexingSplitStore}; use quickwit_metastore::SplitMetadata; diff --git a/quickwit/quickwit-compaction/src/parquet/pipeline.rs b/quickwit/quickwit-compaction/src/parquet/pipeline.rs index 34050b5c7f6..244dbe24470 100644 --- a/quickwit/quickwit-compaction/src/parquet/pipeline.rs +++ b/quickwit/quickwit-compaction/src/parquet/pipeline.rs @@ -29,7 +29,7 @@ use quickwit_indexing::actors::{ }; use quickwit_indexing::models::PublishLock; use quickwit_parquet_engine::merge::metadata_aggregation::merge_parquet_split_metadata; -use quickwit_parquet_engine::merge::policy::ParquetMergeOperation; +use quickwit_parquet_engine::merge::policy::{ParquetMergeOperation, ParquetMergePolicy}; use quickwit_parquet_engine::merge::{MergeConfig, MergeOutputFile, merge_sorted_parquet_files}; use quickwit_parquet_engine::storage::ParquetWriterConfig; use quickwit_proto::compaction::CompactionTaskKind; @@ -231,6 +231,7 @@ pub(crate) struct ParquetCompactionPipeline { metastore: MetastoreServiceClient, max_concurrent_split_uploads: usize, writer_config: ParquetWriterConfig, + merge_policy: Arc, handles: Option, } @@ -244,6 +245,7 @@ impl ParquetCompactionPipeline { metastore: MetastoreServiceClient, max_concurrent_split_uploads: usize, writer_config: ParquetWriterConfig, + merge_policy: Arc, ) -> Self { Self { task_id, @@ -256,6 +258,7 @@ impl ParquetCompactionPipeline { metastore, max_concurrent_split_uploads, writer_config, + merge_policy, handles: None, } } @@ -364,6 +367,7 @@ impl ParquetCompactionPipeline { self.storage.clone(), sequencer_mailbox, self.max_concurrent_split_uploads, + self.merge_policy.clone(), ); let (uploader_mailbox, uploader_handle) = spawn_ctx .spawn_builder() @@ -403,6 +407,9 @@ mod tests { use std::sync::Arc; use std::time::SystemTime; + use quickwit_parquet_engine::merge::policy::{ + ConstWriteAmplificationParquetMergePolicy, ParquetMergePolicyConfig, ParquetSplitMaturity, + }; use quickwit_parquet_engine::split::{ ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TimeRange, }; @@ -424,6 +431,7 @@ mod tests { low_cardinality_tags: Default::default(), high_cardinality_tag_keys: Default::default(), created_at: SystemTime::now(), + maturity: ParquetSplitMaturity::Mature, parquet_file: format!("{split_id}.parquet"), window: Some(0..3600), sort_fields: "metric_name|host|timestamp_secs/V2".to_string(), @@ -437,6 +445,9 @@ mod tests { fn test_status_update_unspawned_pipeline() { let index_uid = IndexUid::for_test("datadog-metrics", 0); let operation = ParquetMergeOperation::new(vec![test_split("s1"), test_split("s2")]); + let merge_policy: Arc = Arc::new( + ConstWriteAmplificationParquetMergePolicy::new(ParquetMergePolicyConfig::default()), + ); let mut pipeline = ParquetCompactionPipeline::new( "task-1".to_string(), TempDirectory::for_test(), @@ -446,6 +457,7 @@ mod tests { MetastoreServiceClient::from_mock(MockMetastoreService::new()), 2, ParquetWriterConfig::default(), + merge_policy, ); let update = pipeline.pipeline_status_update(); diff --git a/quickwit/quickwit-compaction/src/parquet/planner.rs b/quickwit/quickwit-compaction/src/parquet/planner.rs index 4cf905260e5..4a39e50b63f 100644 --- a/quickwit/quickwit-compaction/src/parquet/planner.rs +++ b/quickwit/quickwit-compaction/src/parquet/planner.rs @@ -21,20 +21,19 @@ use anyhow::Result; use quickwit_common::{is_parquet_pipeline_index, is_sketches_index}; use quickwit_indexing::merge_policy::parquet_merge_policy_from_settings; use quickwit_metastore::{ - IndexMetadata, ListIndexesMetadataResponseExt, ListParquetSplitsQuery, - ListParquetSplitsRequestExt, ListParquetSplitsResponseExt, ParquetSplitRecord, + IndexMetadata, ListIndexesMetadataResponseExt, ListParquetSplitsQuery, ParquetSplitRecord, + list_parquet_splits_paginated, }; use quickwit_parquet_engine::merge::policy::{ ParquetMergeOperation, ParquetMergePolicy, ParquetSplitMaturity, }; -use quickwit_parquet_engine::split::ParquetSplitMetadata; +use quickwit_parquet_engine::split::{ParquetSplitKind, ParquetSplitMetadata}; use quickwit_proto::compaction::{ CompactionFailure, CompactionInProgress, CompactionSuccess, CompactionTaskKind, MergeTaskAssignment, }; use quickwit_proto::metastore::{ - ListIndexesMetadataRequest, ListMetricsSplitsRequest, ListSketchSplitsRequest, - MetastoreService, MetastoreServiceClient, + ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; use quickwit_proto::types::{IndexUid, NodeId}; use time::OffsetDateTime; @@ -106,7 +105,7 @@ pub(crate) struct ParquetCompactionPlanner { metastore: MetastoreServiceClient, } -const STARTUP_LOOKBACK: Duration = Duration::from_secs(24 * 60 * 60); +const STARTUP_LOOKBACK: Duration = Duration::from_secs(15 * 60); impl ParquetCompactionPlanner { pub fn new(metastore: MetastoreServiceClient) -> Self { @@ -219,19 +218,16 @@ impl ParquetCompactionPlanner { index_uid: &IndexUid, query: &ListParquetSplitsQuery, ) -> Result> { - let records = if is_sketches_index(&index_uid.index_id) { - let request = ListSketchSplitsRequest::try_from_query(index_uid.clone(), query)?; - self.metastore - .list_sketch_splits(request) - .await? - .deserialize_splits()? + // Page through results — the metastore gRPC response is capped at + // 20 MiB and a single un-paginated list can blow past that as soon + // as a few thousand splits accumulate. + let kind = if is_sketches_index(&index_uid.index_id) { + ParquetSplitKind::Sketches } else { - let request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), query)?; - self.metastore - .list_metrics_splits(request) - .await? - .deserialize_splits()? + ParquetSplitKind::Metrics }; + let records = list_parquet_splits_paginated(self.metastore.clone(), kind, query.clone()) + .await?; Ok(records) } @@ -344,6 +340,7 @@ mod tests { low_cardinality_tags: Default::default(), high_cardinality_tag_keys: Default::default(), created_at: SystemTime::now(), + maturity: ParquetSplitMaturity::Mature, parquet_file: format!("{split_id}.parquet"), window: Some(0..3600), sort_fields: "metric_name|host|timestamp_secs/V2".to_string(), diff --git a/quickwit/quickwit-compaction/src/parquet/state.rs b/quickwit/quickwit-compaction/src/parquet/state.rs index f2694cca727..dc289dca45e 100644 --- a/quickwit/quickwit-compaction/src/parquet/state.rs +++ b/quickwit/quickwit-compaction/src/parquet/state.rs @@ -185,7 +185,7 @@ mod tests { use std::time::{Duration, SystemTime}; use quickwit_parquet_engine::merge::policy::{ - ConstWriteAmplificationParquetMergePolicy, ParquetMergePolicyConfig, + ConstWriteAmplificationParquetMergePolicy, ParquetMergePolicyConfig, ParquetSplitMaturity, }; use quickwit_parquet_engine::split::{ ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TimeRange, @@ -219,6 +219,7 @@ mod tests { low_cardinality_tags: Default::default(), high_cardinality_tag_keys: Default::default(), created_at: SystemTime::now(), + maturity: ParquetSplitMaturity::Mature, parquet_file: format!("{split_id}.parquet"), window: None, sort_fields: "metric_name|host|timestamp_secs/V2".to_string(),