diff --git a/quickwit/quickwit-common/src/lib.rs b/quickwit/quickwit-common/src/lib.rs index 65ce5df8a0b..8cd16e2f676 100644 --- a/quickwit/quickwit-common/src/lib.rs +++ b/quickwit/quickwit-common/src/lib.rs @@ -223,6 +223,19 @@ pub fn is_metrics_index(index_id: &str) -> bool { index_id.starts_with("otel-metrics") || index_id.starts_with("metrics-") } +/// Returns whether the given index ID corresponds to a sketches index. +/// +/// Sketches indexes use the Parquet/DataFusion pipeline with sketch-specific +/// processors and writers. +pub fn is_sketches_index(index_id: &str) -> bool { + index_id.starts_with("sketches-") +} + +/// Returns whether the given index ID uses the Parquet/DataFusion pipeline. +pub fn is_parquet_pipeline_index(index_id: &str) -> bool { + is_metrics_index(index_id) || is_sketches_index(index_id) +} + #[macro_export] macro_rules! ignore_error_kind { ($kind:path, $expr:expr) => { @@ -444,6 +457,21 @@ mod tests { assert!(!is_metrics_index("my-metrics-index")); // Not prefixed } + #[test] + fn test_is_sketches_index() { + assert!(is_sketches_index("sketches-default")); + assert!(!is_sketches_index("otel-metrics")); + assert!(!is_sketches_index("my-index")); + } + + #[test] + fn test_is_parquet_pipeline_index() { + assert!(is_parquet_pipeline_index("otel-metrics")); + assert!(is_parquet_pipeline_index("sketches-default")); + assert!(!is_parquet_pipeline_index("otel-logs-v0_7")); + assert!(!is_parquet_pipeline_index("my-index")); + } + #[test] fn test_parse_bool_lenient() { assert_eq!(parse_bool_lenient("true"), Some(true)); diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs index 9f79032d7c5..e720596cefe 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs @@ -24,7 +24,7 @@ use std::time::{Duration, Instant}; use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use once_cell::sync::OnceCell; -use quickwit_common::is_metrics_index; +use quickwit_common::is_parquet_pipeline_index; use quickwit_common::pretty::PrettySample; use quickwit_config::{FileSourceParams, SourceParams, indexing_pipeline_params_fingerprint}; use quickwit_proto::indexing::{ @@ -218,7 +218,7 @@ fn get_sources_to_schedule(model: &ControlPlaneModel) -> Vec { SourceParams::IngestApi => { // Metrics indexes should use IngestV2 only, not IngestV1. // The ParquetSourceLoader doesn't support IngestV1. - if is_metrics_index(&source_uid.index_uid.index_id) { + if is_parquet_pipeline_index(&source_uid.index_uid.index_id) { continue; } // TODO ingest v1 is scheduled differently diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index cfb2ade9361..ec18ebbec75 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -26,7 +26,7 @@ use quickwit_actors::{ use quickwit_common::metrics::OwnedGaugeGuard; use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; -use quickwit_common::{KillSwitch, is_metrics_index}; +use quickwit_common::{KillSwitch, is_parquet_pipeline_index, is_sketches_index}; use quickwit_config::{IndexingSettings, RetentionPolicy, SourceConfig}; use quickwit_doc_mapper::DocMapper; use quickwit_ingest::IngesterPool; @@ -409,9 +409,12 @@ impl IndexingPipeline { let index_id = &self.params.pipeline_id.index_uid.index_id; - // Route metrics indexes to the Parquet/DataFusion pipeline - if is_metrics_index(index_id) { - return self.spawn_parquet_pipeline(ctx).await; + // Route metrics and sketches indexes to the Parquet/DataFusion pipeline + if is_parquet_pipeline_index(index_id) { + let use_sketch_processors = is_sketches_index(index_id); + return self + .spawn_parquet_pipeline(ctx, use_sketch_processors) + .await; } let source_id = &self.params.pipeline_id.source_id; @@ -585,16 +588,21 @@ impl IndexingPipeline { index=%self.params.pipeline_id.index_uid.index_id, r#gen=self.generation() ))] - async fn spawn_parquet_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { + async fn spawn_parquet_pipeline( + &mut self, + ctx: &ActorContext, + use_sketch_processors: bool, + ) -> anyhow::Result<()> { let index_id = &self.params.pipeline_id.index_uid.index_id; let source_id = &self.params.pipeline_id.source_id; info!( index_id, source_id, + use_sketch_processors, pipeline_uid=%self.params.pipeline_id.pipeline_uid, root_dir=%self.params.indexing_directory.path().display(), - "spawning parquet indexing pipeline for metrics", + "spawning parquet indexing pipeline", ); let (source_mailbox, source_inbox) = ctx @@ -638,11 +646,23 @@ impl IndexingPipeline { // ParquetPackager let writer_config = quickwit_parquet_engine::storage::ParquetWriterConfig::default(); - let split_writer = quickwit_parquet_engine::storage::ParquetSplitWriter::new( + let split_kind = if use_sketch_processors { + quickwit_parquet_engine::split::ParquetSplitKind::Sketches + } else { + quickwit_parquet_engine::split::ParquetSplitKind::Metrics + }; + let sort_order = if use_sketch_processors { + quickwit_parquet_engine::schema::SKETCH_SORT_ORDER + } else { + quickwit_parquet_engine::schema::SORT_ORDER + }; + let split_writer_kind = quickwit_parquet_engine::storage::ParquetSplitWriter::new( + split_kind, writer_config, + sort_order, self.params.indexing_directory.path(), ); - let parquet_packager = ParquetPackager::new(split_writer, parquet_uploader_mailbox); + let parquet_packager = ParquetPackager::new(split_writer_kind, parquet_uploader_mailbox); let (parquet_packager_mailbox, parquet_packager_handle) = ctx .spawn_actor() .set_kill_switch(self.kill_switch.clone()) @@ -664,7 +684,17 @@ impl IndexingPipeline { .spawn(parquet_indexer); // ParquetDocProcessor + let processor = if use_sketch_processors { + crate::actors::parquet_doc_processor::IngestProcessor::Sketches( + quickwit_parquet_engine::ingest::SketchParquetIngestProcessor::new(), + ) + } else { + crate::actors::parquet_doc_processor::IngestProcessor::Metrics( + quickwit_parquet_engine::ingest::ParquetIngestProcessor, + ) + }; let parquet_doc_processor = ParquetDocProcessor::new( + processor, index_id.to_string(), source_id.to_string(), parquet_indexer_mailbox, diff --git a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs index eb51621a30f..57a70c49dd9 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_doc_processor.rs @@ -23,7 +23,9 @@ use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, Qu use quickwit_common::rate_limited_tracing::rate_limited_warn; use quickwit_common::runtimes::RuntimeType; use quickwit_metastore::checkpoint::SourceCheckpointDelta; -use quickwit_parquet_engine::ingest::{IngestError, ParquetIngestProcessor}; +use quickwit_parquet_engine::ingest::{ + IngestError, ParquetIngestProcessor, SketchParquetIngestProcessor, +}; use quickwit_proto::types::{IndexId, SourceId}; use serde::Serialize; use tokio::runtime::Handle; @@ -116,17 +118,32 @@ pub enum ParquetDocProcessorError { Ingest(#[from] IngestError), } -/// ParquetDocProcessor actor that routes Arrow IPC batches to the metrics engine. +/// Enum wrapping the ingest processor variant for metrics vs sketches. +pub enum IngestProcessor { + Metrics(ParquetIngestProcessor), + Sketches(SketchParquetIngestProcessor), +} + +impl IngestProcessor { + fn process_ipc(&self, ipc_bytes: &[u8]) -> Result { + match self { + Self::Metrics(p) => p.process_ipc(ipc_bytes), + Self::Sketches(p) => p.process_ipc(ipc_bytes), + } + } +} + +/// ParquetDocProcessor actor that routes Arrow IPC batches to the parquet engine. /// /// This actor receives RawDocBatch messages containing Arrow IPC data and converts -/// them to RecordBatch using ParquetIngestProcessor. The resulting batches are -/// forwarded to ParquetIndexer for accumulation and split production. +/// them to RecordBatch using the configured ingest processor. The resulting batches +/// are forwarded to ParquetIndexer for accumulation and split production. /// /// Unlike DocProcessor which converts to Tantivy documents, this actor works -/// exclusively with Arrow RecordBatch for high-throughput metrics ingestion. +/// exclusively with Arrow RecordBatch for high-throughput metrics/sketch ingestion. pub struct ParquetDocProcessor { /// Processor for converting Arrow IPC to RecordBatch. - processor: ParquetIngestProcessor, + processor: IngestProcessor, /// Processing counters. counters: ParquetDocProcessorCounters, /// Publish lock for coordinating with sources. @@ -138,11 +155,11 @@ pub struct ParquetDocProcessor { impl ParquetDocProcessor { /// Creates a new ParquetDocProcessor. pub fn new( + processor: IngestProcessor, index_id: IndexId, source_id: SourceId, indexer_mailbox: Mailbox, ) -> Self { - let processor = ParquetIngestProcessor; let counters = ParquetDocProcessorCounters::new(index_id.clone(), source_id.clone()); info!( @@ -403,6 +420,7 @@ mod tests { let (indexer_mailbox, _indexer_inbox) = universe.create_test_mailbox::(); let metrics_doc_processor = ParquetDocProcessor::new( + IngestProcessor::Metrics(ParquetIngestProcessor), "test-metrics-index".to_string(), "test-source".to_string(), indexer_mailbox, @@ -444,6 +462,7 @@ mod tests { let (indexer_mailbox, _indexer_inbox) = universe.create_test_mailbox::(); let metrics_doc_processor = ParquetDocProcessor::new( + IngestProcessor::Metrics(ParquetIngestProcessor), "test-metrics-index".to_string(), "test-source".to_string(), indexer_mailbox, @@ -479,6 +498,7 @@ mod tests { let (indexer_mailbox, _indexer_inbox) = universe.create_test_mailbox::(); let metrics_doc_processor = ParquetDocProcessor::new( + IngestProcessor::Metrics(ParquetIngestProcessor), "test-metrics-index".to_string(), "test-source".to_string(), indexer_mailbox, @@ -549,7 +569,12 @@ mod tests { // Create ParquetPackager let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(writer_config, temp_dir.path()); + let split_writer = ParquetSplitWriter::new( + quickwit_parquet_engine::split::ParquetSplitKind::Metrics, + writer_config, + quickwit_parquet_engine::schema::SORT_ORDER, + temp_dir.path(), + ); let packager = ParquetPackager::new(split_writer, uploader_mailbox); let (packager_mailbox, packager_handle) = universe.spawn_builder().spawn(packager); @@ -564,6 +589,7 @@ mod tests { let (indexer_mailbox, indexer_handle) = universe.spawn_builder().spawn(indexer); let metrics_doc_processor = ParquetDocProcessor::new( + IngestProcessor::Metrics(ParquetIngestProcessor), "test-index".to_string(), "test-source".to_string(), indexer_mailbox, diff --git a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs b/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs index d613fc96003..77dc2a64022 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_e2e_test.rs @@ -12,18 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! End-to-end tests for the metrics pipeline. +//! End-to-end tests for the parquet pipeline //! -//! These tests wire up the full metrics pipeline: -//! ParquetDocProcessor → ParquetIndexer → ParquetUploader → ParquetPublisher +//! These tests wire up the full pipeline: +//! ParquetDocProcessor → ParquetIndexer → ParquetPackager → ParquetUploader → ParquetPublisher use std::sync::Arc; use std::sync::atomic::Ordering; use std::time::Duration; use arrow::array::{ - ArrayRef, DictionaryArray, Float64Array, Int32Array, StringArray, UInt8Array, UInt64Array, + ArrayRef, DictionaryArray, Float64Array, Int16Array, Int32Array, ListArray, StringArray, + UInt8Array, UInt32Array, UInt64Array, }; +use arrow::buffer::OffsetBuffer; use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; use arrow::record_batch::RecordBatch; use bytes::Bytes; @@ -31,6 +33,7 @@ use quickwit_actors::{ActorHandle, Universe}; use quickwit_common::test_utils::wait_until_predicate; use quickwit_metastore::checkpoint::SourceCheckpointDelta; use quickwit_parquet_engine::ingest::record_batch_to_ipc; +use quickwit_parquet_engine::split::ParquetSplitKind; use quickwit_parquet_engine::storage::{ParquetSplitWriter, ParquetWriterConfig}; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_proto::types::IndexUid; @@ -126,7 +129,7 @@ fn create_raw_doc_batch( } // ============================================================================= -// Tests +// Metrics pipeline E2E // ============================================================================= /// Full pipeline test: DocProcessor → Indexer → Uploader → Publisher. @@ -170,9 +173,13 @@ async fn test_metrics_pipeline_e2e() { ); let (uploader_mailbox, _uploader_handle) = universe.spawn_builder().spawn(uploader); - // ParquetPackager between indexer and uploader let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(writer_config, temp_dir.path()); + let split_writer = ParquetSplitWriter::new( + ParquetSplitKind::Metrics, + writer_config, + quickwit_parquet_engine::schema::SORT_ORDER, + temp_dir.path(), + ); let packager = ParquetPackager::new(split_writer, uploader_mailbox); let (packager_mailbox, packager_handle) = universe.spawn_builder().spawn(packager); @@ -186,6 +193,9 @@ async fn test_metrics_pipeline_e2e() { let (indexer_mailbox, indexer_handle) = universe.spawn_builder().spawn(indexer); let doc_processor = ParquetDocProcessor::new( + crate::actors::parquet_doc_processor::IngestProcessor::Metrics( + quickwit_parquet_engine::ingest::ParquetIngestProcessor, + ), "test-metrics-index".to_string(), "test-source".to_string(), indexer_mailbox, @@ -252,17 +262,18 @@ async fn test_metrics_pipeline_e2e() { universe.assert_quit().await; } -/// FileBackedMetastore metrics splits operations (no mocks). -/// -/// Validates stage, publish, list (with time range and metric name filtering). +// ============================================================================= +// FileBackedMetastore metrics operations +// ============================================================================= + #[tokio::test] async fn test_file_backed_metastore_metrics_operations() { use quickwit_config::IndexConfig; use quickwit_metastore::{ - CreateIndexRequestExt, FileBackedMetastore, ListMetricsSplitsQuery, - ListMetricsSplitsRequestExt, ListMetricsSplitsResponseExt, StageMetricsSplitsRequestExt, + CreateIndexRequestExt, FileBackedMetastore, ListParquetSplitsQuery, + ListParquetSplitsRequestExt, ListParquetSplitsResponseExt, StageParquetSplitsRequestExt, }; - use quickwit_parquet_engine::split::{MetricsSplitMetadata, MetricsSplitRecord, TimeRange}; + use quickwit_parquet_engine::split::{ParquetSplitMetadata, ParquetSplitRecord, TimeRange}; use quickwit_proto::metastore::{ CreateIndexRequest, ListMetricsSplitsRequest, MetastoreService, PublishMetricsSplitsRequest, StageMetricsSplitsRequest, @@ -279,7 +290,7 @@ async fn test_file_backed_metastore_metrics_operations() { let create_response = metastore.create_index(create_request).await.unwrap(); let index_uid = create_response.index_uid.unwrap(); - let split1 = MetricsSplitMetadata::builder() + let split1 = ParquetSplitMetadata::builder() .index_uid(index_uid.to_string()) .time_range(TimeRange::new(1000, 1100)) .num_rows(100) @@ -287,7 +298,7 @@ async fn test_file_backed_metastore_metrics_operations() { .add_metric_name("cpu.usage") .build(); - let split2 = MetricsSplitMetadata::builder() + let split2 = ParquetSplitMetadata::builder() .index_uid(index_uid.to_string()) .time_range(TimeRange::new(2000, 2100)) .num_rows(200) @@ -304,11 +315,11 @@ async fn test_file_backed_metastore_metrics_operations() { metastore.stage_metrics_splits(stage_request).await.unwrap(); // Verify staged - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()) + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) .with_split_states(vec!["Staged".to_string()]); let list_request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), &query).unwrap(); let list_response = metastore.list_metrics_splits(list_request).await.unwrap(); - let staged: Vec = list_response.deserialize_splits().unwrap(); + let staged: Vec = list_response.deserialize_splits().unwrap(); assert_eq!(staged.len(), 2); // Publish split1 @@ -325,37 +336,260 @@ async fn test_file_backed_metastore_metrics_operations() { .unwrap(); // Verify published - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()) + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) .with_split_states(vec!["Published".to_string()]); let list_request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), &query).unwrap(); let list_response = metastore.list_metrics_splits(list_request).await.unwrap(); - let published: Vec = list_response.deserialize_splits().unwrap(); + let published: Vec = list_response.deserialize_splits().unwrap(); assert_eq!(published.len(), 1); assert_eq!(published[0].metadata.split_id, split1.split_id); // Time range filtering - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()) + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) .with_split_states(vec!["Published".to_string()]) .with_time_range(1000, 1100); let list_request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), &query).unwrap(); let list_response = metastore.list_metrics_splits(list_request).await.unwrap(); - let in_range: Vec = list_response.deserialize_splits().unwrap(); + let in_range: Vec = list_response.deserialize_splits().unwrap(); assert_eq!(in_range.len(), 1); - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()) + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) .with_split_states(vec!["Published".to_string()]) .with_time_range(5000, 5100); let list_request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), &query).unwrap(); let list_response = metastore.list_metrics_splits(list_request).await.unwrap(); - let out_of_range: Vec = list_response.deserialize_splits().unwrap(); + let out_of_range: Vec = list_response.deserialize_splits().unwrap(); assert_eq!(out_of_range.len(), 0); // Metric name filtering - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()) + let query = ListParquetSplitsQuery::for_index(index_uid.clone()) .with_split_states(vec!["Published".to_string()]) .with_metric_names(vec!["cpu.usage".to_string()]); let list_request = ListMetricsSplitsRequest::try_from_query(index_uid.clone(), &query).unwrap(); let list_response = metastore.list_metrics_splits(list_request).await.unwrap(); - let by_metric: Vec = list_response.deserialize_splits().unwrap(); + let by_metric: Vec = list_response.deserialize_splits().unwrap(); assert_eq!(by_metric.len(), 1); } + +// ============================================================================= +// Sketch pipeline E2E +// ============================================================================= + +fn create_sketch_test_batch( + num_rows: usize, + metric_name: &str, + service: &str, + base_timestamp: u64, +) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("count", DataType::UInt64, false), + Field::new("sum", DataType::Float64, false), + Field::new("min", DataType::Float64, false), + Field::new("max", DataType::Float64, false), + Field::new("flags", DataType::UInt32, false), + Field::new( + "keys", + DataType::List(Arc::new(Field::new("item", DataType::Int16, false))), + false, + ), + Field::new( + "counts", + DataType::List(Arc::new(Field::new("item", DataType::UInt64, false))), + false, + ), + Field::new("service", dict_type, true), + ])); + + let metric_name_arr: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec![metric_name]); + Arc::new(DictionaryArray::::try_new(keys, Arc::new(vals)).unwrap()) + }; + let timestamps: Vec = (0..num_rows).map(|i| base_timestamp + i as u64).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let count: ArrayRef = Arc::new(UInt64Array::from(vec![100u64; num_rows])); + let sum: ArrayRef = Arc::new(Float64Array::from(vec![500.0; num_rows])); + let min_arr: ArrayRef = Arc::new(Float64Array::from(vec![1.0; num_rows])); + let max_arr: ArrayRef = Arc::new(Float64Array::from(vec![200.0; num_rows])); + let flags: ArrayRef = Arc::new(UInt32Array::from(vec![0u32; num_rows])); + + // Build List keys column + let bucket_keys: Vec = vec![100, 200, 300]; + let mut key_offsets = vec![0i32]; + let mut key_values = Vec::new(); + for _ in 0..num_rows { + key_values.extend_from_slice(&bucket_keys); + key_offsets.push(key_values.len() as i32); + } + let keys_arr: ArrayRef = Arc::new(ListArray::new( + Arc::new(Field::new("item", DataType::Int16, false)), + OffsetBuffer::new(key_offsets.into()), + Arc::new(Int16Array::from(key_values)), + None, + )); + + // Build List counts column + let bucket_counts: Vec = vec![50, 30, 20]; + let mut count_offsets = vec![0i32]; + let mut count_values = Vec::new(); + for _ in 0..num_rows { + count_values.extend_from_slice(&bucket_counts); + count_offsets.push(count_values.len() as i32); + } + let counts_arr: ArrayRef = Arc::new(ListArray::new( + Arc::new(Field::new("item", DataType::UInt64, false)), + OffsetBuffer::new(count_offsets.into()), + Arc::new(UInt64Array::from(count_values)), + None, + )); + + let service_arr: ArrayRef = { + let keys = Int32Array::from(vec![0i32; num_rows]); + let vals = StringArray::from(vec![service]); + Arc::new(DictionaryArray::::try_new(keys, Arc::new(vals)).unwrap()) + }; + + RecordBatch::try_new( + schema, + vec![ + metric_name_arr, + timestamp_secs, + count, + sum, + min_arr, + max_arr, + flags, + keys_arr, + counts_arr, + service_arr, + ], + ) + .unwrap() +} + +#[tokio::test] +async fn test_sketch_pipeline_e2e() { + use crate::actors::parquet_doc_processor::IngestProcessor; + + let universe = Universe::with_accelerated_time(); + let temp_dir = tempfile::tempdir().unwrap(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_stage_sketch_splits() + .returning(|_| Ok(EmptyResponse {})); + mock_metastore + .expect_publish_sketch_splits() + .returning(|_| Ok(EmptyResponse {})); + + let metastore_client = + quickwit_proto::metastore::MetastoreServiceClient::from_mock(mock_metastore); + let ram_storage = Arc::new(RamStorage::default()); + + let publisher = ParquetPublisher::new( + PublisherType::ParquetPublisher, + metastore_client.clone(), + None, + None, + ); + let (publisher_mailbox, publisher_handle) = universe.spawn_builder().spawn(publisher); + + let uploader = ParquetUploader::new( + UploaderType::IndexUploader, + metastore_client.clone(), + ram_storage.clone(), + SplitsUpdateMailbox::Publisher(publisher_mailbox), + 4, + ); + let (uploader_mailbox, _uploader_handle) = universe.spawn_builder().spawn(uploader); + + let writer_config = ParquetWriterConfig::default(); + let split_writer = ParquetSplitWriter::new( + ParquetSplitKind::Sketches, + writer_config, + quickwit_parquet_engine::schema::SKETCH_SORT_ORDER, + temp_dir.path(), + ); + let packager = ParquetPackager::new(split_writer, uploader_mailbox); + let (packager_mailbox, packager_handle) = universe.spawn_builder().spawn(packager); + + let indexer = ParquetIndexer::new( + IndexUid::for_test("sketches-test-index", 0), + "test-source".to_string(), + None, + packager_mailbox, + None, + ); + let (indexer_mailbox, indexer_handle) = universe.spawn_builder().spawn(indexer); + + let sketch_processor = IngestProcessor::Sketches( + quickwit_parquet_engine::ingest::SketchParquetIngestProcessor::new(), + ); + let doc_processor = ParquetDocProcessor::new( + sketch_processor, + "sketches-test-index".to_string(), + "test-source".to_string(), + indexer_mailbox, + ); + let (doc_processor_mailbox, doc_processor_handle) = + universe.spawn_builder().spawn(doc_processor); + + // Send sketch batches without force_commit + for i in 0u64..3 { + let batch = create_sketch_test_batch(10, "req.latency", "api", 1000 + i * 10); + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let raw = create_raw_doc_batch(&ipc_bytes, i * 10..(i + 1) * 10, false); + doc_processor_mailbox.send_message(raw).await.unwrap(); + } + + doc_processor_handle.process_pending_and_observe().await; + let indexer_counters = indexer_handle.process_pending_and_observe().await.state; + assert_eq!(indexer_counters.batches_received, 3); + assert_eq!(indexer_counters.rows_indexed, 30); + assert_eq!( + indexer_counters.batches_flushed, 0, + "no flushes without force_commit" + ); + + // Send one more batch with force_commit + let batch = create_sketch_test_batch(5, "req.latency", "api", 2000); + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let raw = create_raw_doc_batch(&ipc_bytes, 30..35, true); + doc_processor_mailbox.send_message(raw).await.unwrap(); + + doc_processor_handle.process_pending_and_observe().await; + indexer_handle.process_pending_and_observe().await; + + wait_for_published_splits(&publisher_handle, 1) + .await + .expect("publisher should have published 1 sketch split"); + + let doc_counters = doc_processor_handle + .process_pending_and_observe() + .await + .state; + assert_eq!(doc_counters.valid_batches, 4); + assert_eq!(doc_counters.valid_rows, 35); + assert_eq!(doc_counters.parse_errors, 0); + + let packager_counters = packager_handle.process_pending_and_observe().await.state; + assert_eq!(packager_counters.splits_produced.load(Ordering::Relaxed), 1); + assert!(packager_counters.bytes_written.load(Ordering::Relaxed) > 0); + + // Verify the parquet file was uploaded to storage + let files: Vec<_> = ram_storage.list_files().await; + assert!( + files + .iter() + .any(|f| f.extension().map(|ext| ext == "parquet").unwrap_or(false)), + "expected a parquet file in storage, found: {files:?}" + ); + + assert_eq!(publisher_handle.last_observation().num_published_splits, 1); + + universe.assert_quit().await; +} diff --git a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs index 254ff2dc719..76f2cf3305c 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_indexer.rs @@ -27,7 +27,7 @@ use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, Qu use quickwit_common::runtimes::RuntimeType; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; use quickwit_parquet_engine::index::{ParquetBatchAccumulator, ParquetIndexingConfig}; -use quickwit_parquet_engine::split::ParquetSplit; +use quickwit_parquet_engine::split::ParquetSplitMetadata; use quickwit_proto::types::{IndexUid, PublishToken, SourceId}; use serde::Serialize; use tokio::runtime::Handle; @@ -86,7 +86,7 @@ pub struct ParquetSplitBatch { /// Index unique identifier for the splits in this batch. pub index_uid: IndexUid, /// The splits produced. - pub splits: Vec, + pub splits: Vec, /// Directory containing the Parquet files referenced by splits. /// The uploader uses this to locate and upload the actual file content. pub output_dir: PathBuf, @@ -593,8 +593,12 @@ mod tests { uploader_mailbox: Mailbox, ) -> (Mailbox, ActorHandle) { let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(writer_config, temp_dir); - + let split_writer = ParquetSplitWriter::new( + quickwit_parquet_engine::split::ParquetSplitKind::Metrics, + writer_config, + quickwit_parquet_engine::schema::SORT_ORDER, + temp_dir, + ); let packager = ParquetPackager::new(split_writer, uploader_mailbox); universe.spawn_builder().spawn(packager) } diff --git a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs index b0950141b10..a168d0a3eb9 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_packager.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_packager.rs @@ -87,8 +87,8 @@ impl ParquetPackagerCounters { /// /// This actor: /// - Receives ParquetBatchForPackager messages from ParquetIndexer -/// - Writes the RecordBatch to a Parquet file via ParquetSplitWriter (sort + encode + compress) -/// - Extracts split metadata (time range, metric names, service names) +/// - Writes the RecordBatch to a Parquet file via the configured split writer +/// - Extracts split metadata (time range, metric names) /// - Forwards the completed ParquetSplitBatch to ParquetUploader /// /// Runs on the blocking runtime since Parquet encoding and file IO are CPU/IO-bound. @@ -184,17 +184,17 @@ impl Handler for ParquetPackager { // Write the batch to a Parquet file match self.split_writer.write_split(&batch, &index_uid_str) { - Ok(split) => { - let size_bytes = split.metadata.size_bytes; + Ok(split_metadata) => { + let size_bytes = split_metadata.size_bytes(); self.counters.record_split(size_bytes); info!( - split_id = %split.metadata.split_id, + split_id = %split_metadata.split_id_str(), num_rows, size_bytes, "ParquetPackager wrote split" ); - vec![split] + vec![split_metadata] } Err(error) => { warn!(error = %error, num_rows, "ParquetPackager failed to write split"); @@ -274,8 +274,12 @@ mod tests { uploader_mailbox: Mailbox, ) -> (Mailbox, ActorHandle) { let writer_config = ParquetWriterConfig::default(); - let split_writer = ParquetSplitWriter::new(writer_config, temp_dir); - + let split_writer = ParquetSplitWriter::new( + quickwit_parquet_engine::split::ParquetSplitKind::Metrics, + writer_config, + quickwit_parquet_engine::schema::SORT_ORDER, + temp_dir, + ); let packager = ParquetPackager::new(split_writer, uploader_mailbox); universe.spawn_builder().spawn(packager) } diff --git a/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs b/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs index 2f5ea943749..651f8915f64 100644 --- a/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/parquet_uploader.rs @@ -27,8 +27,8 @@ use async_trait::async_trait; use once_cell::sync::OnceCell; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, QueueCapacity}; use quickwit_common::spawn_named_task; -use quickwit_metastore::StageMetricsSplitsRequestExt; -use quickwit_parquet_engine::split::MetricsSplitMetadata; +use quickwit_metastore::StageParquetSplitsRequestExt; +use quickwit_parquet_engine::split::{ParquetSplitKind, ParquetSplitMetadata}; use quickwit_proto::metastore::{MetastoreService, MetastoreServiceClient}; use quickwit_storage::Storage; use tokio::sync::{Semaphore, SemaphorePermit}; @@ -44,6 +44,44 @@ use crate::models::ParquetSplitsUpdate; /// Uses same permit pool as indexer uploads. static CONCURRENT_UPLOAD_PERMITS_METRICS: OnceCell = OnceCell::new(); +/// Stage splits in the metastore, dispatching to the correct RPC based on split kind. +async fn stage_splits( + metastore: MetastoreServiceClient, + index_uid: quickwit_proto::types::IndexUid, + splits: &[ParquetSplitMetadata], +) -> anyhow::Result<()> { + if splits.is_empty() { + return Ok(()); + } + + // All splits in a batch must be the same kind (metrics or sketches). + // The pipeline guarantees this since each index uses a single SplitWriterKind. + let kind = splits[0].kind; + debug_assert!( + splits.iter().all(|s| s.kind == kind), + "mixed split types in a single batch" + ); + + match kind { + ParquetSplitKind::Sketches => { + let stage_request = + quickwit_proto::metastore::StageSketchSplitsRequest::try_from_splits_metadata( + index_uid, splits, + )?; + metastore.stage_sketch_splits(stage_request).await?; + } + ParquetSplitKind::Metrics => { + let stage_request = + quickwit_proto::metastore::StageMetricsSplitsRequest::try_from_splits_metadata( + index_uid, splits, + )?; + metastore.stage_metrics_splits(stage_request).await?; + } + } + + Ok(()) +} + /// ParquetUploader actor for staging and uploading metrics splits. /// /// Receives ParquetSplitBatch from ParquetIndexer, stages splits to the metastore, @@ -201,7 +239,6 @@ impl Handler for ParquetUploader { let publish_lock = batch.publish_lock; let publish_token_opt = batch.publish_token_opt; let splits = batch.splits; - debug!( index_uid = %index_uid, num_splits = splits.len(), @@ -221,32 +258,12 @@ impl Handler for ParquetUploader { return; } - // Collect metadata for staging - let splits_metadata: Vec = splits - .iter() - .map(|s| s.metadata.clone()) - .collect(); - - // Stage splits in metastore - let stage_request = match quickwit_proto::metastore::StageMetricsSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - &splits_metadata, - ) { - Ok(req) => req, - Err(e) => { - warn!(error = %e, "failed to create stage metrics splits request"); - // Discard sequencer position on error - if let SplitsUpdateSender::Sequencer(tx) = sender { - let _ = tx.send(SequencerCommand::Discard); - } - kill_switch.kill(); - return; - } - }; + // Stage splits in metastore based on split type + let stage_result = + stage_splits(metastore.clone(), index_uid.clone(), &splits).await; - if let Err(e) = metastore.clone().stage_metrics_splits(stage_request).await { - warn!(error = %e, "failed to stage metrics splits"); - // Discard sequencer position on error + if let Err(e) = stage_result { + warn!(error = %e, "failed to stage splits"); if let SplitsUpdateSender::Sequencer(tx) = sender { let _ = tx.send(SequencerCommand::Discard); } @@ -254,16 +271,18 @@ impl Handler for ParquetUploader { return; } - counters.num_staged_splits.fetch_add(splits_metadata.len() as u64, Ordering::SeqCst); + counters + .num_staged_splits + .fetch_add(splits.len() as u64, Ordering::SeqCst); info!( index_uid = %index_uid, - num_splits = splits_metadata.len(), - "staged metrics splits in metastore" + num_splits = splits.len(), + "staged splits in metastore" ); // Upload Parquet files to storage for split in &splits { - let parquet_file = split.metadata.parquet_filename(); + let parquet_file = split.parquet_filename(); // Read the local Parquet file from output_dir let local_path = output_dir.join(&parquet_file); let file_content = match tokio::fs::read(&local_path).await { @@ -272,7 +291,7 @@ impl Handler for ParquetUploader { warn!( error = %e, local_path = %local_path.display(), - split_id = %split.metadata.split_id, + split_id = %split.split_id_str(), parquet_file = %parquet_file, "failed to read local parquet file" ); @@ -289,13 +308,10 @@ impl Handler for ParquetUploader { let payload: Box = Box::new(file_content); // Upload to S3 using the filename directly (matches logs pipeline) - if let Err(e) = split_store - .put(Path::new(&parquet_file), payload) - .await - { + if let Err(e) = split_store.put(Path::new(&parquet_file), payload).await { warn!( error = %e, - split_id = %split.metadata.split_id, + split_id = %split.split_id_str(), parquet_file = %parquet_file, "failed to upload parquet file" ); @@ -319,7 +335,7 @@ impl Handler for ParquetUploader { } debug!( - split_id = %split.metadata.split_id, + split_id = %split.split_id_str(), parquet_file = %parquet_file, file_size = file_size, "uploaded parquet file to storage" @@ -329,7 +345,7 @@ impl Handler for ParquetUploader { // Create ParquetSplitsUpdate and send downstream let update = ParquetSplitsUpdate { index_uid, - new_splits: splits_metadata, + new_splits: splits, replaced_split_ids: Vec::new(), // No merging yet checkpoint_delta_opt: Some(checkpoint_delta), publish_lock, @@ -367,7 +383,7 @@ impl Handler for ParquetUploader { mod tests { use quickwit_actors::{ObservationType, Universe}; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_parquet_engine::split::{MetricsSplitMetadata, ParquetSplit, TimeRange}; + use quickwit_parquet_engine::split::{ParquetSplitMetadata, TimeRange}; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; use quickwit_proto::types::IndexUid; use quickwit_storage::RamStorage; @@ -376,22 +392,26 @@ mod tests { use crate::actors::Sequencer; use crate::models::PublishLock; - fn create_test_metrics_split(index_uid: &str, split_id: &str) -> ParquetSplit { - let metadata = MetricsSplitMetadata::builder() + fn create_test_metrics_split(index_uid: &str, split_id: &str) -> ParquetSplitMetadata { + ParquetSplitMetadata::builder() .index_uid(index_uid) - .split_id(quickwit_parquet_engine::split::SplitId::new(split_id)) + .split_id(quickwit_parquet_engine::split::ParquetSplitId::new( + split_id, + )) .time_range(TimeRange::new(1000, 2000)) .num_rows(100) .size_bytes(1024) - .build(); - ParquetSplit::new(metadata) + .build() } /// Create placeholder parquet files in the temp directory for testing. /// The uploader expects to read these files from output_dir. - fn create_placeholder_parquet_files(temp_dir: &std::path::Path, splits: &[ParquetSplit]) { + fn create_placeholder_parquet_files( + temp_dir: &std::path::Path, + splits: &[ParquetSplitMetadata], + ) { for split in splits { - let parquet_filename = split.metadata.parquet_filename(); + let parquet_filename = split.parquet_filename(); let file_path = temp_dir.join(&parquet_filename); // Write minimal valid content (actual parquet not needed for staging test) std::fs::write(&file_path, b"placeholder parquet content") @@ -695,7 +715,7 @@ mod tests { // The inbox contains typed messages, we need to access the ParquetSplitsUpdate if let Some(update) = msg.downcast_ref::() { for split in &update.new_splits { - received_split_ids.push(split.split_id.as_str().to_string()); + received_split_ids.push(split.split_id_str().to_string()); } } } diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index c4ec75b1a8a..115097a533f 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -18,7 +18,8 @@ use fail::fail_point; use quickwit_actors::{Actor, ActorContext, Handler, Mailbox, QueueCapacity}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_proto::metastore::{ - MetastoreService, MetastoreServiceClient, PublishMetricsSplitsRequest, PublishSplitsRequest, + MetastoreService, MetastoreServiceClient, PublishMetricsSplitsRequest, + PublishSketchSplitsRequest, PublishSplitsRequest, }; use serde::Serialize; use tracing::{info, instrument}; @@ -256,19 +257,39 @@ impl Handler for Publisher { let index_checkpoint_delta_json_opt = serialize_checkpoint_delta(&checkpoint_delta_opt)?; let split_ids: Vec = new_splits .iter() - .map(|split| split.split_id.as_str().to_string()) + .map(|split| split.split_id_str().to_string()) .collect(); + + // Determine whether these are metrics or sketch splits for publishing. + let is_sketch = new_splits + .first() + .map(|s| s.kind == quickwit_parquet_engine::split::ParquetSplitKind::Sketches) + .unwrap_or(false); + if let Some(_guard) = publish_lock.acquire().await { - let publish_request = PublishMetricsSplitsRequest { - index_uid: Some(index_uid.clone()), - staged_split_ids: split_ids.clone(), - replaced_split_ids: replaced_split_ids.clone(), - index_checkpoint_delta_json_opt, - publish_token_opt: publish_token_opt.clone(), - }; - ctx.protect_future(self.metastore.publish_metrics_splits(publish_request)) - .await - .context("failed to publish metrics splits")?; + if is_sketch { + let publish_request = PublishSketchSplitsRequest { + index_uid: Some(index_uid.clone()), + staged_split_ids: split_ids.clone(), + replaced_split_ids: replaced_split_ids.clone(), + index_checkpoint_delta_json_opt, + publish_token_opt: publish_token_opt.clone(), + }; + ctx.protect_future(self.metastore.publish_sketch_splits(publish_request)) + .await + .context("failed to publish sketch splits")?; + } else { + let publish_request = PublishMetricsSplitsRequest { + index_uid: Some(index_uid.clone()), + staged_split_ids: split_ids.clone(), + replaced_split_ids: replaced_split_ids.clone(), + index_checkpoint_delta_json_opt, + publish_token_opt: publish_token_opt.clone(), + }; + ctx.protect_future(self.metastore.publish_metrics_splits(publish_request)) + .await + .context("failed to publish metrics splits")?; + } } else { info!( split_ids=?split_ids, @@ -560,7 +581,7 @@ mod tests { mod parquet_publisher_tests { use quickwit_actors::Universe; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_parquet_engine::split::{MetricsSplitMetadata, SplitId, TimeRange}; + use quickwit_parquet_engine::split::{ParquetSplitId, ParquetSplitMetadata, TimeRange}; use quickwit_proto::metastore::{EmptyResponse, MetastoreServiceClient, MockMetastoreService}; use quickwit_proto::types::IndexUid; use tracing::Span; @@ -568,10 +589,10 @@ mod parquet_publisher_tests { use super::*; use crate::models::PublishLock; - fn create_test_metrics_split_metadata(index_uid: &str, split_id: &str) -> MetricsSplitMetadata { - MetricsSplitMetadata::builder() + fn create_test_metrics_split_metadata(index_uid: &str, split_id: &str) -> ParquetSplitMetadata { + ParquetSplitMetadata::builder() .index_uid(index_uid) - .split_id(SplitId::new(split_id)) + .split_id(ParquetSplitId::new(split_id)) .time_range(TimeRange::new(1000, 2000)) .num_rows(100) .size_bytes(1024) diff --git a/quickwit/quickwit-indexing/src/models/parquet_splits_update.rs b/quickwit/quickwit-indexing/src/models/parquet_splits_update.rs index 5342743546a..6dfef0bfe43 100644 --- a/quickwit/quickwit-indexing/src/models/parquet_splits_update.rs +++ b/quickwit/quickwit-indexing/src/models/parquet_splits_update.rs @@ -18,7 +18,7 @@ use std::fmt; use itertools::Itertools; use quickwit_metastore::checkpoint::IndexCheckpointDelta; -use quickwit_parquet_engine::split::MetricsSplitMetadata; +use quickwit_parquet_engine::split::ParquetSplitMetadata; use quickwit_proto::types::{IndexUid, PublishToken}; use tracing::Span; @@ -26,13 +26,13 @@ use crate::models::PublishLock; /// Message sent by ParquetUploader to downstream actors after staging and uploading. /// -/// This is analogous to `SplitsUpdate` but uses `MetricsSplitMetadata` instead of -/// `SplitMetadata`. +/// This is analogous to `SplitsUpdate` but uses `ParquetSplitMetadata` to support +/// both metrics and sketch splits (distinguished by the `kind` field). pub struct ParquetSplitsUpdate { /// Index unique identifier. pub index_uid: IndexUid, - /// The staged and uploaded splits. - pub new_splits: Vec, + /// The staged and uploaded splits (metrics or sketches). + pub new_splits: Vec, /// Split IDs being replaced (for merges, typically empty for ingest). pub replaced_split_ids: Vec, /// Checkpoint delta covering the data in these splits. @@ -50,7 +50,7 @@ impl fmt::Debug for ParquetSplitsUpdate { let new_split_ids: String = self .new_splits .iter() - .map(|split| split.split_id.as_str()) + .map(|split| split.split_id_str()) .join(","); f.debug_struct("ParquetSplitsUpdate") .field("index_uid", &self.index_uid) diff --git a/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.down.sql b/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.down.sql new file mode 100644 index 00000000000..bf7bd80cca5 --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.down.sql @@ -0,0 +1,8 @@ +DROP INDEX IF EXISTS idx_sketch_splits_tag_host; +DROP INDEX IF EXISTS idx_sketch_splits_tag_region; +DROP INDEX IF EXISTS idx_sketch_splits_tag_datacenter; +DROP INDEX IF EXISTS idx_sketch_splits_tag_env; +DROP INDEX IF EXISTS idx_sketch_splits_tag_service; +DROP INDEX IF EXISTS idx_sketch_splits_metric_names; +DROP INDEX IF EXISTS idx_sketch_splits_time; +DROP TABLE IF EXISTS sketch_splits; diff --git a/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.up.sql b/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.up.sql new file mode 100644 index 00000000000..eafb828e313 --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/postgresql/27_create-sketch-splits.up.sql @@ -0,0 +1,47 @@ +-- Sketch splits table for DDSketch data. +-- Mirrors metrics_splits structure for two-tier pruning. +CREATE TABLE IF NOT EXISTS sketch_splits ( + -- Identity + split_id VARCHAR(50) PRIMARY KEY, + split_state VARCHAR(30) NOT NULL, + index_uid VARCHAR(282) NOT NULL, + + -- Temporal pruning + time_range_start BIGINT NOT NULL, + time_range_end BIGINT NOT NULL, + + -- Metric name pruning + metric_names TEXT[] NOT NULL, + + -- Low-cardinality tag pruning + tag_service TEXT[], + tag_env TEXT[], + tag_datacenter TEXT[], + tag_region TEXT[], + tag_host TEXT[], + + -- High-cardinality tag keys (values stored in Parquet bloom filters) + high_cardinality_tag_keys TEXT[] NOT NULL DEFAULT '{}', + + -- Planning metadata + num_rows BIGINT NOT NULL, + size_bytes BIGINT NOT NULL, + + -- Full metadata + split_metadata_json TEXT NOT NULL, + + -- Timestamps + create_timestamp TIMESTAMP NOT NULL DEFAULT (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), + update_timestamp TIMESTAMP NOT NULL DEFAULT (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), + publish_timestamp TIMESTAMP, + + FOREIGN KEY(index_uid) REFERENCES indexes(index_uid) +); + +CREATE INDEX idx_sketch_splits_time ON sketch_splits (time_range_start, time_range_end); +CREATE INDEX idx_sketch_splits_metric_names ON sketch_splits USING GIN (metric_names); +CREATE INDEX idx_sketch_splits_tag_service ON sketch_splits USING GIN (tag_service); +CREATE INDEX idx_sketch_splits_tag_env ON sketch_splits USING GIN (tag_env); +CREATE INDEX idx_sketch_splits_tag_datacenter ON sketch_splits USING GIN (tag_datacenter); +CREATE INDEX idx_sketch_splits_tag_region ON sketch_splits USING GIN (tag_region); +CREATE INDEX idx_sketch_splits_tag_host ON sketch_splits USING GIN (tag_host); diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index e8fffeb7962..08a0798ffb2 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -45,10 +45,10 @@ pub use metastore::postgres::PostgresqlMetastore; pub use metastore::{ AddSourceRequestExt, CreateIndexRequestExt, CreateIndexResponseExt, IndexMetadata, IndexMetadataResponseExt, IndexesMetadataResponseExt, ListIndexesMetadataResponseExt, - ListMetricsSplitsQuery, ListMetricsSplitsRequestExt, ListMetricsSplitsResponseExt, + ListParquetSplitsQuery, ListParquetSplitsRequestExt, ListParquetSplitsResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, - MetastoreServiceStreamSplitsExt, PublishMetricsSplitsRequestExt, PublishSplitsRequestExt, - StageMetricsSplitsRequestExt, StageSplitsRequestExt, UpdateIndexRequestExt, + MetastoreServiceStreamSplitsExt, PublishParquetSplitsRequestExt, PublishSplitsRequestExt, + StageParquetSplitsRequestExt, StageSplitsRequestExt, UpdateIndexRequestExt, UpdateSourceRequestExt, file_backed, }; pub use metastore_factory::{MetastoreFactory, UnsupportedMetastore}; diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 44dc4f58282..68e43f286f5 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -21,19 +21,21 @@ use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, DeleteIndexTemplatesRequest, DeleteMetricsSplitsRequest, DeleteQuery, DeleteShardsRequest, - DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, - FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, - GetClusterIdentityResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, - IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataRequest, IndexesMetadataResponse, - LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, + DeleteShardsResponse, DeleteSketchSplitsRequest, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, + GetClusterIdentityRequest, GetClusterIdentityResponse, GetIndexTemplateRequest, + GetIndexTemplateResponse, IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataRequest, + IndexesMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, + ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListMetricsSplitsRequest, ListMetricsSplitsResponse, - ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, - ListStaleSplitsRequest, MarkMetricsSplitsForDeletionRequest, MarkSplitsForDeletionRequest, - MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, - OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishMetricsSplitsRequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, StageMetricsSplitsRequest, + ListShardsRequest, ListShardsResponse, ListSketchSplitsRequest, ListSketchSplitsResponse, + ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, + MarkMetricsSplitsForDeletionRequest, MarkSketchSplitsForDeletionRequest, + MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, MetastoreServiceClient, + MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, + PublishMetricsSplitsRequest, PublishSketchSplitsRequest, PublishSplitsRequest, + ResetSourceCheckpointRequest, StageMetricsSplitsRequest, StageSketchSplitsRequest, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; @@ -330,4 +332,41 @@ impl MetastoreService for ControlPlaneMetastore { ) -> MetastoreResult { self.metastore.delete_metrics_splits(request).await } + + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> MetastoreResult { + self.metastore.stage_sketch_splits(request).await + } + + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> MetastoreResult { + self.metastore.publish_sketch_splits(request).await + } + + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> MetastoreResult { + self.metastore.list_sketch_splits(request).await + } + + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> MetastoreResult { + self.metastore + .mark_sketch_splits_for_deletion(request) + .await + } + + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> MetastoreResult { + self.metastore.delete_sketch_splits(request).await + } } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs index 6fd5ce244be..fc76e92cdf3 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs @@ -28,7 +28,7 @@ use quickwit_common::pretty::PrettySample; use quickwit_config::{ DocMapping, IndexingSettings, IngestSettings, RetentionPolicy, SearchSettings, SourceConfig, }; -use quickwit_parquet_engine::split::{MetricsSplitMetadata, MetricsSplitState}; +use quickwit_parquet_engine::split::{ParquetSplitMetadata, SplitState as ParquetSplitState}; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, IndexStats, ListShardsSubrequest, @@ -44,20 +44,25 @@ use tracing::{info, warn}; use super::MutationOccurred; use crate::checkpoint::IndexCheckpointDelta; -use crate::metastore::{ListMetricsSplitsQuery, SortBy, use_shard_api}; +use crate::metastore::{ListParquetSplitsQuery, SortBy, use_shard_api}; use crate::{IndexMetadata, ListSplitsQuery, Split, SplitMetadata, SplitState, split_tag_filter}; -/// A stored metrics split with its state. +/// A stored parquet split (metrics or sketch) with its state. #[derive(Clone, Debug, Serialize, Deserialize)] -pub(crate) struct StoredMetricsSplit { +pub(crate) struct StoredParquetSplit { /// The split metadata. - pub metadata: MetricsSplitMetadata, + pub metadata: ParquetSplitMetadata, /// The split state. - pub state: MetricsSplitState, + pub state: ParquetSplitState, /// Update timestamp (Unix epoch seconds). pub update_timestamp: i64, } +/// Backwards-compatible alias. +pub(crate) type StoredMetricsSplit = StoredParquetSplit; +/// Backwards-compatible alias. +pub(crate) type StoredSketchSplit = StoredParquetSplit; + /// A `FileBackedIndex` object carries an index metadata and its split metadata. // This struct is meant to be used only within the [`FileBackedMetastore`]. The public visibility is #[derive(Clone, Debug, Serialize, Deserialize)] @@ -74,6 +79,8 @@ pub(crate) struct FileBackedIndex { delete_tasks: Vec, /// Metrics splits (for metrics pipeline). metrics_splits: HashMap, + /// Sketch splits (for DDSketch pipeline). + sketch_splits: HashMap, /// Stamper. stamper: Stamper, /// Flag used to avoid polling the metastore if @@ -163,6 +170,7 @@ impl From for FileBackedIndex { per_source_shards, delete_tasks: Default::default(), metrics_splits: Default::default(), + sketch_splits: Default::default(), stamper: Default::default(), recently_modified: false, discarded: false, @@ -186,16 +194,24 @@ impl FileBackedIndex { per_source_shards: HashMap, delete_tasks: Vec, ) -> Self { - Self::new_with_metrics_splits(metadata, splits, per_source_shards, delete_tasks, vec![]) + Self::new_with_metrics_splits( + metadata, + splits, + per_source_shards, + delete_tasks, + vec![], + vec![], + ) } - /// Constructor with metrics splits. + /// Constructor with metrics and sketch splits. pub fn new_with_metrics_splits( metadata: IndexMetadata, splits: Vec, per_source_shards: HashMap, delete_tasks: Vec, metrics_splits: Vec, + sketch_splits: Vec, ) -> Self { let last_opstamp = delete_tasks .iter() @@ -210,12 +226,17 @@ impl FileBackedIndex { .into_iter() .map(|split| (split.metadata.split_id.to_string(), split)) .collect(); + let sketch_splits = sketch_splits + .into_iter() + .map(|split| (split.metadata.split_id.as_str().to_string(), split)) + .collect(); Self { metadata, splits, per_source_shards, delete_tasks, metrics_splits, + sketch_splits, stamper: Stamper::new(last_opstamp), recently_modified: false, discarded: false, @@ -730,39 +751,14 @@ impl FileBackedIndex { .try_apply_delta(checkpoint_delta.source_delta, publish_token) } - // Metrics Splits API + // Parquet Splits API /// Stages metrics splits. pub(crate) fn stage_metrics_splits( &mut self, - splits_metadata: Vec, + splits_metadata: Vec, ) -> MetastoreResult { - if splits_metadata.is_empty() { - return Ok(false); - } - let now = OffsetDateTime::now_utc().unix_timestamp(); - for metadata in splits_metadata { - let split_id = metadata.split_id.to_string(); - - // Reject if the split already exists in a non-Staged state. - if let Some(existing) = self.metrics_splits.get(&split_id) - && existing.state != MetricsSplitState::Staged - { - let entity = EntityKind::Split { - split_id: split_id.clone(), - }; - let message = "split is not in Staged state".to_string(); - return Err(MetastoreError::FailedPrecondition { entity, message }); - } - - let stored = StoredMetricsSplit { - metadata, - state: MetricsSplitState::Staged, - update_timestamp: now, - }; - self.metrics_splits.insert(split_id, stored); - } - Ok(true) + stage_parquet_splits(&mut self.metrics_splits, splits_metadata) } /// Publishes metrics splits (transitions from Staged to Published). @@ -773,7 +769,86 @@ impl FileBackedIndex { checkpoint_delta_opt: Option, publish_token_opt: Option, ) -> MetastoreResult { - // Apply checkpoint delta to index metadata, matching publish_splits behavior. + self.apply_checkpoint_delta(checkpoint_delta_opt, publish_token_opt)?; + publish_parquet_splits( + &mut self.metrics_splits, + staged_split_ids, + replaced_split_ids, + ) + } + + /// Lists metrics splits matching the query. + pub(crate) fn list_metrics_splits( + &self, + query: &ListParquetSplitsQuery, + ) -> Vec { + list_parquet_splits(&self.metrics_splits, query) + } + + /// Marks metrics splits for deletion. + pub(crate) fn mark_metrics_splits_for_deletion( + &mut self, + split_ids: &[String], + ) -> MetastoreResult { + mark_parquet_splits_for_deletion(&mut self.metrics_splits, split_ids) + } + + /// Deletes metrics splits (must be MarkedForDeletion). + pub(crate) fn delete_metrics_splits(&mut self, split_ids: &[String]) -> MetastoreResult { + delete_parquet_splits(&mut self.metrics_splits, split_ids) + } + + /// Stages sketch splits. + pub(crate) fn stage_sketch_splits( + &mut self, + splits_metadata: Vec, + ) -> MetastoreResult { + stage_parquet_splits(&mut self.sketch_splits, splits_metadata) + } + + /// Publishes sketch splits (transitions from Staged to Published). + pub(crate) fn publish_sketch_splits( + &mut self, + staged_split_ids: &[String], + replaced_split_ids: &[String], + checkpoint_delta_opt: Option, + publish_token_opt: Option, + ) -> MetastoreResult { + self.apply_checkpoint_delta(checkpoint_delta_opt, publish_token_opt)?; + publish_parquet_splits( + &mut self.sketch_splits, + staged_split_ids, + replaced_split_ids, + ) + } + + /// Lists sketch splits matching the query. + pub(crate) fn list_sketch_splits( + &self, + query: &ListParquetSplitsQuery, + ) -> Vec { + list_parquet_splits(&self.sketch_splits, query) + } + + /// Marks sketch splits for deletion. + pub(crate) fn mark_sketch_splits_for_deletion( + &mut self, + split_ids: &[String], + ) -> MetastoreResult { + mark_parquet_splits_for_deletion(&mut self.sketch_splits, split_ids) + } + + /// Deletes sketch splits (must be MarkedForDeletion). + pub(crate) fn delete_sketch_splits(&mut self, split_ids: &[String]) -> MetastoreResult { + delete_parquet_splits(&mut self.sketch_splits, split_ids) + } + + /// Apply checkpoint delta for parquet split publishing + fn apply_checkpoint_delta( + &mut self, + checkpoint_delta_opt: Option, + publish_token_opt: Option, + ) -> MetastoreResult<()> { if let Some(checkpoint_delta) = checkpoint_delta_opt { let source_id = checkpoint_delta.source_id.clone(); let source = self.metadata.sources.get(&source_id).ok_or_else(|| { @@ -810,100 +885,138 @@ impl FileBackedIndex { })?; } } + Ok(()) + } +} - let now = OffsetDateTime::now_utc().unix_timestamp(); +fn stage_parquet_splits( + splits_map: &mut HashMap, + splits_metadata: Vec, +) -> MetastoreResult { + if splits_metadata.is_empty() { + return Ok(false); + } + let now = OffsetDateTime::now_utc().unix_timestamp(); + for metadata in splits_metadata { + let split_id = metadata.split_id.as_str().to_string(); - // Verify all staged splits exist and are in Staged state - for split_id in staged_split_ids { - let split = self.metrics_splits.get(split_id).ok_or_else(|| { - MetastoreError::NotFound(EntityKind::Splits { - split_ids: vec![split_id.clone()], - }) - })?; - if split.state != MetricsSplitState::Staged { - return Err(MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { - split_ids: vec![split_id.clone()], - }, - message: format!("split {} is not in Staged state", split_id), - }); - } + if let Some(existing) = splits_map.get(&split_id) + && existing.state != ParquetSplitState::Staged + { + let entity = EntityKind::Split { + split_id: split_id.clone(), + }; + let message = "split is not in Staged state".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); } - // Transition staged splits to Published - for split_id in staged_split_ids { - if let Some(split) = self.metrics_splits.get_mut(split_id) { - split.state = MetricsSplitState::Published; - split.update_timestamp = now; - } - } + let stored = StoredParquetSplit { + metadata, + state: ParquetSplitState::Staged, + update_timestamp: now, + }; + splits_map.insert(split_id, stored); + } + Ok(true) +} - // Mark replaced splits for deletion - for split_id in replaced_split_ids { - if let Some(split) = self.metrics_splits.get_mut(split_id) { - split.state = MetricsSplitState::MarkedForDeletion; - split.update_timestamp = now; - } +fn publish_parquet_splits( + splits_map: &mut HashMap, + staged_split_ids: &[String], + replaced_split_ids: &[String], +) -> MetastoreResult { + let now = OffsetDateTime::now_utc().unix_timestamp(); + + // Verify all staged splits exist and are in Staged state + for split_id in staged_split_ids { + let split = splits_map.get(split_id).ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Splits { + split_ids: vec![split_id.clone()], + }) + })?; + if split.state != ParquetSplitState::Staged { + return Err(MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { + split_ids: vec![split_id.clone()], + }, + message: format!("split {} is not in Staged state", split_id), + }); } + } - Ok(true) + // Transition staged splits to Published + for split_id in staged_split_ids { + if let Some(split) = splits_map.get_mut(split_id) { + split.state = ParquetSplitState::Published; + split.update_timestamp = now; + } } - /// Lists metrics splits matching the query. - pub(crate) fn list_metrics_splits( - &self, - query: &ListMetricsSplitsQuery, - ) -> Vec { - self.metrics_splits - .values() - .filter(|split| metrics_split_matches_query(split, query)) - .cloned() - .collect() + // Mark replaced splits for deletion + for split_id in replaced_split_ids { + if let Some(split) = splits_map.get_mut(split_id) { + split.state = ParquetSplitState::MarkedForDeletion; + split.update_timestamp = now; + } } - /// Marks metrics splits for deletion. - pub(crate) fn mark_metrics_splits_for_deletion( - &mut self, - split_ids: &[String], - ) -> MetastoreResult { - let now = OffsetDateTime::now_utc().unix_timestamp(); - let mut mutated = false; - for split_id in split_ids { - if let Some(split) = self.metrics_splits.get_mut(split_id) { - split.state = MetricsSplitState::MarkedForDeletion; - split.update_timestamp = now; - mutated = true; - } + Ok(true) +} + +fn list_parquet_splits( + splits_map: &HashMap, + query: &ListParquetSplitsQuery, +) -> Vec { + splits_map + .values() + .filter(|split| parquet_split_matches_query(split, query)) + .cloned() + .collect() +} + +fn mark_parquet_splits_for_deletion( + splits_map: &mut HashMap, + split_ids: &[String], +) -> MetastoreResult { + let now = OffsetDateTime::now_utc().unix_timestamp(); + let mut mutated = false; + for split_id in split_ids { + if let Some(split) = splits_map.get_mut(split_id) { + split.state = ParquetSplitState::MarkedForDeletion; + split.update_timestamp = now; + mutated = true; } - Ok(mutated) } + Ok(mutated) +} - /// Deletes metrics splits (must be MarkedForDeletion). - pub(crate) fn delete_metrics_splits(&mut self, split_ids: &[String]) -> MetastoreResult { - for split_id in split_ids { - if let Some(split) = self.metrics_splits.get(split_id) - && split.state != MetricsSplitState::MarkedForDeletion - { - return Err(MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { - split_ids: vec![split_id.clone()], - }, - message: format!("split {} is not marked for deletion", split_id), - }); - } +fn delete_parquet_splits( + splits_map: &mut HashMap, + split_ids: &[String], +) -> MetastoreResult { + for split_id in split_ids { + if let Some(split) = splits_map.get(split_id) + && split.state != ParquetSplitState::MarkedForDeletion + { + return Err(MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { + split_ids: vec![split_id.clone()], + }, + message: format!("split {} is not marked for deletion", split_id), + }); } - let mut mutated = false; - for split_id in split_ids { - if self.metrics_splits.remove(split_id).is_some() { - mutated = true; - } + } + let mut mutated = false; + for split_id in split_ids { + if splits_map.remove(split_id).is_some() { + mutated = true; } - Ok(mutated) } + Ok(mutated) } -/// Checks if a metrics split matches the query criteria. -fn metrics_split_matches_query(split: &StoredMetricsSplit, query: &ListMetricsSplitsQuery) -> bool { +/// Checks if a parquet split matches the query criteria. +fn parquet_split_matches_query(split: &StoredParquetSplit, query: &ListParquetSplitsQuery) -> bool { // Filter by state if !query.split_states.is_empty() { let state_str = split.state.as_str(); diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/serialize.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/serialize.rs index dc7293f7b72..9d5377db8fe 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/serialize.rs @@ -20,7 +20,7 @@ use quickwit_proto::metastore::SourceType; use quickwit_proto::types::{DocMappingUid, SourceId}; use serde::{Deserialize, Serialize}; -use super::StoredMetricsSplit; +use super::StoredParquetSplit; use super::shards::Shards; use crate::file_backed::file_backed_index::FileBackedIndex; use crate::metastore::DeleteTask; @@ -71,7 +71,10 @@ pub(crate) struct FileBackedIndexV0_8 { delete_tasks: Vec, /// Metrics splits (for metrics pipeline). #[serde(default, skip_serializing_if = "Vec::is_empty")] - metrics_splits: Vec, + metrics_splits: Vec, + /// Sketch splits (for DDSketch pipeline). + #[serde(default, skip_serializing_if = "Vec::is_empty")] + sketch_splits: Vec, } impl From for FileBackedIndexV0_8 { @@ -106,12 +109,18 @@ impl From for FileBackedIndexV0_8 { .into_values() .sorted_by_key(|split| split.update_timestamp) .collect(); + let sketch_splits = index + .sketch_splits + .into_values() + .sorted_by_key(|split| split.update_timestamp) + .collect(); Self { metadata: index.metadata, splits, shards, delete_tasks, metrics_splits, + sketch_splits, } } } @@ -145,6 +154,7 @@ impl From for FileBackedIndex { per_source_shards, index.delete_tasks, index.metrics_splits, + index.sketch_splits, ) } } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index 00791488e65..68a7d8dc02f 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -43,22 +43,25 @@ use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, DeleteIndexTemplatesRequest, DeleteMetricsSplitsRequest, DeleteQuery, DeleteShardsRequest, - DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, - EntityKind, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, - GetClusterIdentityRequest, GetClusterIdentityResponse, GetIndexTemplateRequest, - GetIndexTemplateResponse, IndexMetadataFailure, IndexMetadataFailureReason, - IndexMetadataRequest, IndexMetadataResponse, IndexTemplateMatch, IndexesMetadataRequest, - IndexesMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, - ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexStatsRequest, ListIndexStatsResponse, - ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, - ListIndexesMetadataResponse, ListMetricsSplitsRequest, ListMetricsSplitsResponse, - ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, - ListStaleSplitsRequest, MarkMetricsSplitsForDeletionRequest, MarkSplitsForDeletionRequest, - MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, + DeleteShardsResponse, DeleteSketchSplitsRequest, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, EntityKind, FindIndexTemplateMatchesRequest, + FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, GetClusterIdentityResponse, + GetIndexTemplateRequest, GetIndexTemplateResponse, IndexMetadataFailure, + IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, IndexTemplateMatch, + IndexesMetadataRequest, IndexesMetadataResponse, LastDeleteOpstampRequest, + LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, + ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, + ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListMetricsSplitsRequest, ListMetricsSplitsResponse, ListShardsRequest, ListShardsResponse, + ListSketchSplitsRequest, ListSketchSplitsResponse, ListSplitsRequest, ListSplitsResponse, + ListStaleSplitsRequest, MarkMetricsSplitsForDeletionRequest, + MarkSketchSplitsForDeletionRequest, MarkSplitsForDeletionRequest, MetastoreError, + MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishMetricsSplitsRequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, StageMetricsSplitsRequest, - StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, - UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, serde_utils, + PublishSketchSplitsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, + StageMetricsSplitsRequest, StageSketchSplitsRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, serde_utils, }; use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::Storage; @@ -76,10 +79,10 @@ use self::state::MetastoreState; use self::store_operations::{delete_index, index_exists, load_index, put_index}; use super::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, - IndexesMetadataResponseExt, ListIndexesMetadataResponseExt, ListMetricsSplitsRequestExt, - ListMetricsSplitsResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, - PublishMetricsSplitsRequestExt, PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, - StageMetricsSplitsRequestExt, StageSplitsRequestExt, UpdateIndexRequestExt, + IndexesMetadataResponseExt, ListIndexesMetadataResponseExt, ListParquetSplitsRequestExt, + ListParquetSplitsResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, + PublishParquetSplitsRequestExt, PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, + StageParquetSplitsRequestExt, StageSplitsRequestExt, UpdateIndexRequestExt, UpdateSourceRequestExt, }; use crate::checkpoint::IndexCheckpointDelta; @@ -1286,10 +1289,8 @@ impl MetastoreService for FileBackedMetastore { &self, request: StageMetricsSplitsRequest, ) -> MetastoreResult { - use quickwit_parquet_engine::split::MetricsSplitMetadata; - let index_uid = request.index_uid().clone(); - let splits_metadata: Vec = request.deserialize_splits_metadata()?; + let splits_metadata = request.deserialize_splits_metadata()?; if splits_metadata.is_empty() { return Ok(EmptyResponse {}); @@ -1345,7 +1346,7 @@ impl MetastoreService for FileBackedMetastore { &self, request: ListMetricsSplitsRequest, ) -> MetastoreResult { - use quickwit_parquet_engine::split::MetricsSplitRecord; + use quickwit_parquet_engine::split::ParquetSplitRecord; let index_uid = request.index_uid().clone(); let query = request.deserialize_query()?; @@ -1354,10 +1355,9 @@ impl MetastoreService for FileBackedMetastore { .read(&index_uid, |index| Ok(index.list_metrics_splits(&query))) .await?; - // Convert StoredMetricsSplit to MetricsSplitRecord for the response - let split_records: Vec = stored_splits + let split_records: Vec = stored_splits .into_iter() - .map(|s| MetricsSplitRecord { + .map(|s| ParquetSplitRecord { state: s.state, update_timestamp: s.update_timestamp, metadata: s.metadata, @@ -1414,6 +1414,136 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let splits_metadata = request.deserialize_splits_metadata()?; + + if splits_metadata.is_empty() { + return Ok(EmptyResponse {}); + } + + self.mutate(&index_uid, |index| { + let mutated = index.stage_sketch_splits(splits_metadata)?; + if mutated { + Ok(MutationOccurred::Yes(())) + } else { + Ok(MutationOccurred::No(())) + } + }) + .await?; + + Ok(EmptyResponse {}) + } + + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> MetastoreResult { + let index_checkpoint_delta: Option = + request.deserialize_index_checkpoint()?; + let index_uid = request.index_uid().clone(); + let staged_split_ids = request.staged_split_ids; + let replaced_split_ids = request.replaced_split_ids; + let publish_token_opt = request.publish_token_opt; + + if staged_split_ids.is_empty() && replaced_split_ids.is_empty() { + return Ok(EmptyResponse {}); + } + + self.mutate(&index_uid, |index| { + let mutated = index.publish_sketch_splits( + &staged_split_ids, + &replaced_split_ids, + index_checkpoint_delta, + publish_token_opt, + )?; + if mutated { + Ok(MutationOccurred::Yes(())) + } else { + Ok(MutationOccurred::No(())) + } + }) + .await?; + + Ok(EmptyResponse {}) + } + + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> MetastoreResult { + use quickwit_parquet_engine::split::ParquetSplitRecord; + + let index_uid = request.index_uid().clone(); + let query = request.deserialize_query()?; + + let stored_splits = self + .read(&index_uid, |index| Ok(index.list_sketch_splits(&query))) + .await?; + + let split_records: Vec = stored_splits + .into_iter() + .map(|s| ParquetSplitRecord { + state: s.state, + update_timestamp: s.update_timestamp, + metadata: s.metadata, + }) + .collect(); + + ListSketchSplitsResponse::try_from_splits(&split_records) + } + + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let split_ids = request.split_ids; + + if split_ids.is_empty() { + return Ok(EmptyResponse {}); + } + + self.mutate(&index_uid, |index| { + let mutated = index.mark_sketch_splits_for_deletion(&split_ids)?; + if mutated { + Ok(MutationOccurred::Yes(())) + } else { + Ok(MutationOccurred::No(())) + } + }) + .await?; + + Ok(EmptyResponse {}) + } + + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let split_ids = request.split_ids; + + if split_ids.is_empty() { + return Ok(EmptyResponse {}); + } + + self.mutate(&index_uid, |index| { + let mutated = index.delete_sketch_splits(&split_ids)?; + if mutated { + Ok(MutationOccurred::Yes(())) + } else { + Ok(MutationOccurred::No(())) + } + }) + .await?; + + Ok(EmptyResponse {}) + } } impl MetastoreServiceExt for FileBackedMetastore {} diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index ddfee25afab..cc0de579382 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -33,15 +33,15 @@ use quickwit_config::{ SearchSettings, SourceConfig, SourceParams, }; use quickwit_doc_mapper::tag_pruning::TagFilterAst; -use quickwit_parquet_engine::split::{MetricsSplitMetadata, MetricsSplitRecord}; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteTask, IndexMetadataFailure, IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataResponse, ListIndexesMetadataResponse, ListMetricsSplitsRequest, ListMetricsSplitsResponse, - ListSplitsRequest, ListSplitsResponse, MetastoreError, MetastoreResult, MetastoreService, - MetastoreServiceClient, MetastoreServiceStream, PublishMetricsSplitsRequest, - PublishSplitsRequest, StageMetricsSplitsRequest, StageSplitsRequest, UpdateIndexRequest, - UpdateSourceRequest, serde_utils, + ListSketchSplitsRequest, ListSketchSplitsResponse, ListSplitsRequest, ListSplitsResponse, + MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, + MetastoreServiceStream, PublishMetricsSplitsRequest, PublishSketchSplitsRequest, + PublishSplitsRequest, StageMetricsSplitsRequest, StageSketchSplitsRequest, StageSplitsRequest, + UpdateIndexRequest, UpdateSourceRequest, serde_utils, }; use quickwit_proto::types::{IndexUid, NodeId, SplitId}; use serde::{Deserialize, Serialize}; @@ -50,9 +50,9 @@ use time::OffsetDateTime; use crate::checkpoint::IndexCheckpointDelta; use crate::{Split, SplitMetadata, SplitState}; -/// Query parameters for listing metrics splits. +/// Query parameters for listing parquet splits (metrics or sketches). #[derive(Debug, Clone, Default, Serialize, Deserialize)] -pub struct ListMetricsSplitsQuery { +pub struct ListParquetSplitsQuery { /// Index UID to filter by (required). pub index_uid: IndexUid, /// Split states to include. @@ -79,8 +79,8 @@ pub struct ListMetricsSplitsQuery { pub limit: Option, } -impl ListMetricsSplitsQuery { - /// Creates a query for all splits in an index. +impl ListParquetSplitsQuery { + /// Creates a query for all published splits in an index. pub fn for_index(index_uid: impl Into) -> Self { Self { index_uid: index_uid.into(), @@ -649,22 +649,6 @@ impl PublishSplitsRequestExt for PublishSplitsRequest { } } -/// Helper trait for [`PublishMetricsSplitsRequest`] to deserialize its payload. -pub trait PublishMetricsSplitsRequestExt { - /// Deserializes the `index_checkpoint_delta_json_opt` field of a - /// [`PublishMetricsSplitsRequest`] into an [`Option`]. - fn deserialize_index_checkpoint(&self) -> MetastoreResult>; -} - -impl PublishMetricsSplitsRequestExt for PublishMetricsSplitsRequest { - fn deserialize_index_checkpoint(&self) -> MetastoreResult> { - self.index_checkpoint_delta_json_opt - .as_ref() - .map(|value| serde_utils::from_json_str(value)) - .transpose() - } -} - #[async_trait] impl ListSplitsResponseExt for ListSplitsResponse { fn empty() -> Self { @@ -710,121 +694,162 @@ impl ListSplitsResponseExt for ListSplitsResponse { } // ===================================================== -// Metrics Splits Extension Traits +// Parquet Splits Extension Traits (unified via macros) // ===================================================== -/// Helper trait to build a [`StageMetricsSplitsRequest`]. -pub trait StageMetricsSplitsRequestExt { - /// Creates a new [`StageMetricsSplitsRequest`] from an index UID and split metadata. +use quickwit_parquet_engine::split::{ParquetSplitMetadata, ParquetSplitRecord}; + +/// Helper trait to build stage requests for parquet splits. +pub trait StageParquetSplitsRequestExt: Sized { + /// Creates a new stage request from an index UID and split metadata. fn try_from_splits_metadata( index_uid: impl Into, - splits_metadata: &[MetricsSplitMetadata], - ) -> MetastoreResult; + splits_metadata: &[ParquetSplitMetadata], + ) -> MetastoreResult; /// Deserializes the splits metadata. - fn deserialize_splits_metadata(&self) -> MetastoreResult>; + fn deserialize_splits_metadata(&self) -> MetastoreResult>; } -impl StageMetricsSplitsRequestExt for StageMetricsSplitsRequest { - fn try_from_splits_metadata( - index_uid: impl Into, - splits_metadata: &[MetricsSplitMetadata], - ) -> MetastoreResult { - let splits_metadata_json = splits_metadata - .iter() - .map(serde_utils::to_json_str) - .collect::>>()?; - Ok(StageMetricsSplitsRequest { - index_uid: Some(index_uid.into()), - splits_metadata_json, - }) - } +macro_rules! impl_stage_parquet_splits_ext { + ($request_type:ty) => { + impl StageParquetSplitsRequestExt for $request_type { + fn try_from_splits_metadata( + index_uid: impl Into, + splits_metadata: &[ParquetSplitMetadata], + ) -> MetastoreResult { + let splits_metadata_json = splits_metadata + .iter() + .map(serde_utils::to_json_str) + .collect::>>()?; + Ok(Self { + index_uid: Some(index_uid.into()), + splits_metadata_json, + }) + } + + fn deserialize_splits_metadata(&self) -> MetastoreResult> { + self.splits_metadata_json + .iter() + .map(|s| serde_utils::from_json_str(s)) + .collect() + } + } + }; +} +impl_stage_parquet_splits_ext!(StageMetricsSplitsRequest); +impl_stage_parquet_splits_ext!(StageSketchSplitsRequest); - fn deserialize_splits_metadata(&self) -> MetastoreResult> { - self.splits_metadata_json - .iter() - .map(|s| serde_utils::from_json_str(s)) - .collect() - } +/// Helper trait to deserialize publish requests for parquet splits. +pub trait PublishParquetSplitsRequestExt { + /// Deserializes the `index_checkpoint_delta_json_opt` field into an + /// [`Option`]. + fn deserialize_index_checkpoint(&self) -> MetastoreResult>; +} + +macro_rules! impl_publish_parquet_splits_ext { + ($request_type:ty) => { + impl PublishParquetSplitsRequestExt for $request_type { + fn deserialize_index_checkpoint( + &self, + ) -> MetastoreResult> { + self.index_checkpoint_delta_json_opt + .as_ref() + .map(|value| serde_utils::from_json_str(value)) + .transpose() + } + } + }; } +impl_publish_parquet_splits_ext!(PublishMetricsSplitsRequest); +impl_publish_parquet_splits_ext!(PublishSketchSplitsRequest); -/// Helper trait to build a [`ListMetricsSplitsRequest`]. -pub trait ListMetricsSplitsRequestExt { - /// Creates a new [`ListMetricsSplitsRequest`] from an index UID. - fn for_index(index_uid: impl Into) -> MetastoreResult; +/// Helper trait to build list requests for parquet splits. +pub trait ListParquetSplitsRequestExt: Sized { + /// Creates a new list request from an index UID. + fn for_index(index_uid: impl Into) -> MetastoreResult; - /// Creates a new [`ListMetricsSplitsRequest`] from an index UID and query. + /// Creates a new list request from an index UID and query. fn try_from_query( index_uid: impl Into, - query: &ListMetricsSplitsQuery, - ) -> MetastoreResult; + query: &ListParquetSplitsQuery, + ) -> MetastoreResult; /// Deserializes the `query_json` field. - fn deserialize_query(&self) -> MetastoreResult; + fn deserialize_query(&self) -> MetastoreResult; } -impl ListMetricsSplitsRequestExt for ListMetricsSplitsRequest { - fn for_index(index_uid: impl Into) -> MetastoreResult { - let index_uid = index_uid.into(); - let query = ListMetricsSplitsQuery::for_index(index_uid.clone()); - Self::try_from_query(index_uid, &query) - } - - fn try_from_query( - index_uid: impl Into, - query: &ListMetricsSplitsQuery, - ) -> MetastoreResult { - let query_json = serde_utils::to_json_str(query)?; - Ok(ListMetricsSplitsRequest { - index_uid: Some(index_uid.into()), - query_json, - }) - } - - fn deserialize_query(&self) -> MetastoreResult { - serde_utils::from_json_str(&self.query_json) - } +macro_rules! impl_list_parquet_splits_request_ext { + ($request_type:ty) => { + impl ListParquetSplitsRequestExt for $request_type { + fn for_index(index_uid: impl Into) -> MetastoreResult { + let index_uid = index_uid.into(); + let query = ListParquetSplitsQuery::for_index(index_uid.clone()); + Self::try_from_query(index_uid, &query) + } + + fn try_from_query( + index_uid: impl Into, + query: &ListParquetSplitsQuery, + ) -> MetastoreResult { + let query_json = serde_utils::to_json_str(query)?; + Ok(Self { + index_uid: Some(index_uid.into()), + query_json, + }) + } + + fn deserialize_query(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.query_json) + } + } + }; } +impl_list_parquet_splits_request_ext!(ListMetricsSplitsRequest); +impl_list_parquet_splits_request_ext!(ListSketchSplitsRequest); -/// Helper trait to build and deserialize [`ListMetricsSplitsResponse`]. -pub trait ListMetricsSplitsResponseExt { +/// Helper trait to build and deserialize list responses for parquet splits. +pub trait ListParquetSplitsResponseExt: Sized { /// Creates an empty response. fn empty() -> Self; /// Creates a response from a list of splits. - fn try_from_splits(splits: &[MetricsSplitRecord]) - -> MetastoreResult; + fn try_from_splits(splits: &[ParquetSplitRecord]) -> MetastoreResult; /// Deserializes the splits in the response. - fn deserialize_splits(&self) -> MetastoreResult>; + fn deserialize_splits(&self) -> MetastoreResult>; } -impl ListMetricsSplitsResponseExt for ListMetricsSplitsResponse { - fn empty() -> Self { - Self { - splits_serialized_json: Vec::new(), +macro_rules! impl_list_parquet_splits_response_ext { + ($response_type:ty) => { + impl ListParquetSplitsResponseExt for $response_type { + fn empty() -> Self { + Self { + splits_serialized_json: Vec::new(), + } + } + + fn try_from_splits(splits: &[ParquetSplitRecord]) -> MetastoreResult { + let splits_serialized_json = splits + .iter() + .map(serde_utils::to_json_str) + .collect::>>()?; + Ok(Self { + splits_serialized_json, + }) + } + + fn deserialize_splits(&self) -> MetastoreResult> { + self.splits_serialized_json + .iter() + .map(|s| serde_utils::from_json_str(s)) + .collect() + } } - } - - fn try_from_splits( - splits: &[MetricsSplitRecord], - ) -> MetastoreResult { - let splits_serialized_json = splits - .iter() - .map(serde_utils::to_json_str) - .collect::>>()?; - Ok(ListMetricsSplitsResponse { - splits_serialized_json, - }) - } - - fn deserialize_splits(&self) -> MetastoreResult> { - self.splits_serialized_json - .iter() - .map(|s| serde_utils::from_json_str(s)) - .collect() - } + }; } +impl_list_parquet_splits_response_ext!(ListMetricsSplitsResponse); +impl_list_parquet_splits_response_ext!(ListSketchSplitsResponse); #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] /// A query builder for listing splits within the metastore. diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index d459ceb243b..b68db779b96 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -31,9 +31,10 @@ use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, CreateIndexTemplateRequest, DeleteIndexRequest, DeleteIndexTemplatesRequest, DeleteMetricsSplitsRequest, DeleteQuery, DeleteShardsRequest, - DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, - EntityKind, FindIndexTemplateMatchesRequest, FindIndexTemplateMatchesResponse, - GetClusterIdentityRequest, GetClusterIdentityResponse, GetIndexTemplateRequest, + DeleteShardsResponse, DeleteSketchSplitsRequest, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, EntityKind, FindIndexTemplateMatchesRequest, + FindIndexTemplateMatchesResponse, GetClusterIdentityRequest, GetClusterIdentityResponse, + GetIndexRoutingTableRequest, GetIndexRoutingTableResponse, GetIndexTemplateRequest, GetIndexTemplateResponse, IndexMetadataFailure, IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataResponse, IndexStats, IndexTemplateMatch, IndexesMetadataRequest, IndexesMetadataResponse, LastDeleteOpstampRequest, @@ -41,14 +42,16 @@ use quickwit_proto::metastore::{ ListIndexStatsRequest, ListIndexStatsResponse, ListIndexTemplatesRequest, ListIndexTemplatesResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListMetricsSplitsRequest, ListMetricsSplitsResponse, ListShardsRequest, ListShardsResponse, - ListShardsSubresponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, - MarkMetricsSplitsForDeletionRequest, MarkSplitsForDeletionRequest, MetastoreError, + ListShardsSubresponse, ListSketchSplitsRequest, ListSketchSplitsResponse, ListSplitsRequest, + ListSplitsResponse, ListStaleSplitsRequest, MarkMetricsSplitsForDeletionRequest, + MarkSketchSplitsForDeletionRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, - PublishMetricsSplitsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, SplitStats, - StageMetricsSplitsRequest, StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, - UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, - serde_utils, + PublishMetricsSplitsRequest, PublishSketchSplitsRequest, PublishSplitsRequest, + ResetSourceCheckpointRequest, SetIndexRoutingTableRequest, SplitStats, + StageMetricsSplitsRequest, StageSketchSplitsRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, serde_utils, }; use quickwit_proto::types::{IndexId, IndexUid, Position, PublishToken, ShardId, SourceId}; use sea_query::{Alias, Asterisk, Expr, Func, PostgresQueryBuilder, Query, UnionType}; @@ -75,8 +78,8 @@ use crate::file_backed::MutationOccurred; use crate::metastore::postgres::model::Shards; use crate::metastore::postgres::utils::split_maturity_timestamp; use crate::metastore::{ - IndexesMetadataResponseExt, ListMetricsSplitsResponseExt, PublishMetricsSplitsRequestExt, - PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, StageMetricsSplitsRequestExt, + IndexesMetadataResponseExt, ListParquetSplitsResponseExt, PublishParquetSplitsRequestExt, + PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, StageParquetSplitsRequestExt, UpdateSourceRequestExt, use_shard_api, }; use crate::{ @@ -368,2006 +371,2165 @@ where Ok(index_metadata) } -#[async_trait] -impl MetastoreService for PostgresqlMetastore { - async fn check_connectivity(&self) -> anyhow::Result<()> { - self.connection_pool.acquire().await?; - Ok(()) - } - - fn endpoints(&self) -> Vec { - vec![self.uri.clone()] - } +use quickwit_parquet_engine::split::{ + InsertableParquetSplit, ParquetSplitKind, ParquetSplitMetadata, ParquetSplitRecord, + PgParquetSplit, SplitState as ParquetSplitState, +}; - // Index API: - // - `create_index` - // - `update_index` - // - `index_metadata` - // - `indexes_metadata` - // - `list_indexes_metadata` +use crate::metastore::ListParquetSplitsQuery; - #[instrument(skip(self))] - async fn create_index( +impl PostgresqlMetastore { + /// Shared implementation for staging parquet splits (metrics or sketches). + async fn stage_parquet_splits_impl( &self, - request: CreateIndexRequest, - ) -> MetastoreResult { - let index_config = request.deserialize_index_config()?; - let mut index_metadata = IndexMetadata::new(index_config); + kind: ParquetSplitKind, + index_uid: IndexUid, + splits_metadata: Vec, + ) -> MetastoreResult { + if splits_metadata.is_empty() { + return Ok(EmptyResponse {}); + } - let source_configs = request.deserialize_source_configs()?; + let table_name = kind.table_name(); + let label = kind.label(); - for source_config in source_configs { - index_metadata.add_source(source_config)?; + let mut split_ids = Vec::with_capacity(splits_metadata.len()); + let mut split_states = Vec::with_capacity(splits_metadata.len()); + let mut index_uids = Vec::with_capacity(splits_metadata.len()); + let mut time_range_starts = Vec::with_capacity(splits_metadata.len()); + let mut time_range_ends = Vec::with_capacity(splits_metadata.len()); + let mut metric_names_json = Vec::with_capacity(splits_metadata.len()); + let mut tag_service_json = Vec::with_capacity(splits_metadata.len()); + let mut tag_env_json = Vec::with_capacity(splits_metadata.len()); + let mut tag_datacenter_json = Vec::with_capacity(splits_metadata.len()); + let mut tag_region_json = Vec::with_capacity(splits_metadata.len()); + let mut tag_host_json = Vec::with_capacity(splits_metadata.len()); + let mut high_card_tag_keys_json = Vec::with_capacity(splits_metadata.len()); + let mut num_rows_list = Vec::with_capacity(splits_metadata.len()); + let mut size_bytes_list = Vec::with_capacity(splits_metadata.len()); + let mut split_metadata_jsons = Vec::with_capacity(splits_metadata.len()); + + for metadata in &splits_metadata { + let insertable = + InsertableParquetSplit::from_metadata(metadata, ParquetSplitState::Staged) + .map_err(|err| MetastoreError::JsonSerializeError { + struct_name: "ParquetSplitMetadata".to_string(), + message: err.to_string(), + })?; + + split_ids.push(insertable.split_id); + split_states.push(insertable.split_state); + index_uids.push(insertable.index_uid); + time_range_starts.push(insertable.time_range_start); + time_range_ends.push(insertable.time_range_end); + let json_err = |err: serde_json::Error| MetastoreError::JsonSerializeError { + struct_name: "ParquetSplitMetadata".to_string(), + message: err.to_string(), + }; + metric_names_json + .push(serde_json::to_string(&insertable.metric_names).map_err(json_err)?); + tag_service_json + .push(serde_json::to_string(&insertable.tag_service).map_err(json_err)?); + tag_env_json.push(serde_json::to_string(&insertable.tag_env).map_err(json_err)?); + tag_datacenter_json + .push(serde_json::to_string(&insertable.tag_datacenter).map_err(json_err)?); + tag_region_json.push(serde_json::to_string(&insertable.tag_region).map_err(json_err)?); + tag_host_json.push(serde_json::to_string(&insertable.tag_host).map_err(json_err)?); + high_card_tag_keys_json.push( + serde_json::to_string(&insertable.high_cardinality_tag_keys).map_err(json_err)?, + ); + num_rows_list.push(insertable.num_rows); + size_bytes_list.push(insertable.size_bytes); + split_metadata_jsons.push(insertable.split_metadata_json); } - let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; - sqlx::query( - "INSERT INTO indexes (index_uid, index_id, index_metadata_json) VALUES ($1, $2, $3)", - ) - .bind(index_metadata.index_uid.to_string()) - .bind(&index_metadata.index_uid.index_id) - .bind(&index_metadata_json) - .execute(&self.connection_pool) - .await - .map_err(|sqlx_error| convert_sqlx_err(index_metadata.index_id(), sqlx_error))?; + info!( + index_uid = %index_uid, + num_splits = split_ids.len(), + "staging {label} splits" + ); - let response = CreateIndexResponse { - index_uid: index_metadata.index_uid.into(), - index_metadata_json, - }; - Ok(response) - } + // table_name is a &'static str from ParquetSplitKind, safe for SQL interpolation. + let stage_query = format!( + r#" + INSERT INTO {table_name} ( + split_id, + split_state, + index_uid, + time_range_start, + time_range_end, + metric_names, + tag_service, + tag_env, + tag_datacenter, + tag_region, + tag_host, + high_cardinality_tag_keys, + num_rows, + size_bytes, + split_metadata_json, + create_timestamp, + update_timestamp + ) + SELECT + split_id, + split_state, + index_uid, + time_range_start, + time_range_end, + ARRAY(SELECT json_array_elements_text(metric_names_json::json)), + CASE WHEN tag_service_json::text = 'null' THEN NULL + ELSE ARRAY(SELECT json_array_elements_text(tag_service_json::json)) END, + CASE WHEN tag_env_json::text = 'null' THEN NULL + ELSE ARRAY(SELECT json_array_elements_text(tag_env_json::json)) END, + CASE WHEN tag_datacenter_json::text = 'null' THEN NULL + ELSE ARRAY(SELECT json_array_elements_text(tag_datacenter_json::json)) END, + CASE WHEN tag_region_json::text = 'null' THEN NULL + ELSE ARRAY(SELECT json_array_elements_text(tag_region_json::json)) END, + CASE WHEN tag_host_json::text = 'null' THEN NULL + ELSE ARRAY(SELECT json_array_elements_text(tag_host_json::json)) END, + ARRAY(SELECT json_array_elements_text(high_cardinality_tag_keys_json::json)), + num_rows, + size_bytes, + split_metadata_json, + (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), + (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + FROM UNNEST( + $1::text[], + $2::text[], + $3::text[], + $4::bigint[], + $5::bigint[], + $6::text[], + $7::text[], + $8::text[], + $9::text[], + $10::text[], + $11::text[], + $12::text[], + $13::bigint[], + $14::bigint[], + $15::text[] + ) AS staged( + split_id, + split_state, + index_uid, + time_range_start, + time_range_end, + metric_names_json, + tag_service_json, + tag_env_json, + tag_datacenter_json, + tag_region_json, + tag_host_json, + high_cardinality_tag_keys_json, + num_rows, + size_bytes, + split_metadata_json + ) + ON CONFLICT (split_id) DO UPDATE + SET + split_state = EXCLUDED.split_state, + time_range_start = EXCLUDED.time_range_start, + time_range_end = EXCLUDED.time_range_end, + metric_names = EXCLUDED.metric_names, + tag_service = EXCLUDED.tag_service, + tag_env = EXCLUDED.tag_env, + tag_datacenter = EXCLUDED.tag_datacenter, + tag_region = EXCLUDED.tag_region, + tag_host = EXCLUDED.tag_host, + high_cardinality_tag_keys = EXCLUDED.high_cardinality_tag_keys, + num_rows = EXCLUDED.num_rows, + size_bytes = EXCLUDED.size_bytes, + split_metadata_json = EXCLUDED.split_metadata_json, + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + WHERE {table_name}.split_state = 'Staged' + RETURNING split_id + "#, + ); - async fn update_index( - &self, - request: UpdateIndexRequest, - ) -> MetastoreResult { - let doc_mapping = request.deserialize_doc_mapping()?; - let indexing_settings = request.deserialize_indexing_settings()?; - let ingest_settings = request.deserialize_ingest_settings()?; - let search_settings = request.deserialize_search_settings()?; - let retention_policy_opt = request.deserialize_retention_policy()?; + let split_ids_snapshot = split_ids.clone(); + let index_id_for_err = index_uid.index_id.clone(); + let upserted_split_ids: Vec = { + let mut tx: Transaction<'_, Postgres> = self.connection_pool.begin().await?; + let tx_ref = &mut tx; + let op_result: MetastoreResult> = async { + sqlx::query_scalar(&stage_query) + .bind(&split_ids) + .bind(&split_states) + .bind(&index_uids) + .bind(&time_range_starts) + .bind(&time_range_ends) + .bind(&metric_names_json) + .bind(&tag_service_json) + .bind(&tag_env_json) + .bind(&tag_datacenter_json) + .bind(&tag_region_json) + .bind(&tag_host_json) + .bind(&high_card_tag_keys_json) + .bind(&num_rows_list) + .bind(&size_bytes_list) + .bind(&split_metadata_jsons) + .fetch_all(tx_ref.as_mut()) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_id_for_err, sqlx_error)) + } + .await; + match &op_result { + Ok(_) => { + debug!("committing transaction"); + tx.commit().await?; + } + Err(error) => { + rate_limited_error!(limit_per_min = 60, error=%error, "failed to stage {} splits, rolling transaction back", label); + tx.rollback().await?; + } + } + op_result + }?; - let index_uid: IndexUid = request.index_uid().clone(); - let updated_index_metadata = run_with_tx!(self.connection_pool, tx, "update index", { - mutate_index_metadata::(tx, index_uid, |index_metadata| { - let mutation_occurred = index_metadata.update_index_config( - doc_mapping, - indexing_settings, - ingest_settings, - search_settings, - retention_policy_opt, - )?; - Ok(MutationOccurred::from(mutation_occurred)) - }) - .await - })?; - IndexMetadataResponse::try_from_index_metadata(&updated_index_metadata) + if upserted_split_ids.len() != split_ids_snapshot.len() { + let failed_split_ids: Vec = split_ids_snapshot + .into_iter() + .filter(|split_id| !upserted_split_ids.contains(split_id)) + .collect(); + let entity = EntityKind::Splits { + split_ids: failed_split_ids.clone(), + }; + let message = "splits are not in the Staged state".to_string(); + warn!( + %index_uid, + failed_split_ids = ?failed_split_ids, + "failed to stage some {label} splits" + ); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + + info!( + %index_uid, + num_staged = upserted_split_ids.len(), + "staged {label} splits successfully" + ); + Ok(EmptyResponse {}) } - #[instrument(skip(self))] - async fn index_metadata( + /// Shared implementation for publishing parquet splits. + async fn publish_parquet_splits_impl( &self, - request: IndexMetadataRequest, - ) -> MetastoreResult { - let pg_index_opt = if let Some(index_uid) = &request.index_uid { - index_opt_for_uid(&self.connection_pool, index_uid.clone(), false).await? - } else if let Some(index_id) = &request.index_id { - index_opt(&self.connection_pool, index_id, false).await? - } else { - let message = "invalid request: neither `index_id` nor `index_uid` is set".to_string(); - return Err(MetastoreError::Internal { - message, - cause: "".to_string(), - }); - }; - let index_metadata = pg_index_opt - .ok_or(MetastoreError::NotFound(EntityKind::Index { - index_id: request - .into_index_id() - .expect("`index_id` or `index_uid` should be set"), - }))? - .index_metadata()?; - let response = IndexMetadataResponse::try_from_index_metadata(&index_metadata)?; - Ok(response) - } + kind: ParquetSplitKind, + index_uid: IndexUid, + staged_split_ids: Vec, + replaced_split_ids: Vec, + checkpoint_delta_opt: Option, + publish_token_opt: Option, + ) -> MetastoreResult { + let table_name = kind.table_name(); + let label = kind.label(); - #[instrument(skip(self))] - async fn indexes_metadata( - &self, - request: IndexesMetadataRequest, - ) -> MetastoreResult { - const INDEXES_METADATA_QUERY: &str = include_str!("queries/indexes_metadata.sql"); + info!( + %index_uid, + staged_splits = staged_split_ids.len(), + replaced_splits = replaced_split_ids.len(), + "publishing {label} splits" + ); - let num_subrequests = request.subrequests.len(); + { + let mut tx: Transaction<'_, Postgres> = self.connection_pool.begin().await?; + let tx_ref = &mut tx; + let op_result: MetastoreResult = async { + let mut index_metadata_inner = + index_metadata(tx_ref, &index_uid.index_id, true).await?; + if index_metadata_inner.index_uid != index_uid { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id.clone(), + })); + } + let index_uid_inner = index_metadata_inner.index_uid.clone(); + + if let Some(checkpoint_delta) = checkpoint_delta_opt { + let source_id = checkpoint_delta.source_id.clone(); + let source = index_metadata_inner + .sources + .get(&source_id) + .ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Source { + index_id: index_uid_inner.index_id.to_string(), + source_id: source_id.to_string(), + }) + })?; - if num_subrequests == 0 { - return Ok(Default::default()); - } - let mut index_ids: Vec = Vec::new(); - let mut index_uids: Vec = Vec::with_capacity(num_subrequests); - let mut failures: Vec = Vec::new(); + if use_shard_api(&source.source_params) { + let publish_token = publish_token_opt.clone().ok_or_else(|| { + let message = format!( + "publish token is required for publishing splits for source \ + `{source_id}`" + ); + MetastoreError::InvalidArgument { message } + })?; + try_apply_delta_v2( + tx_ref, + &index_uid_inner, + &source_id, + checkpoint_delta.source_delta, + publish_token, + ) + .await?; + } else { + index_metadata_inner + .checkpoint + .try_apply_delta(checkpoint_delta) + .map_err(|error| { + let entity = EntityKind::CheckpointDelta { + index_id: index_uid_inner.index_id.to_string(), + source_id, + }; + let message = error.to_string(); + MetastoreError::FailedPrecondition { entity, message } + })?; + } + } + let index_metadata_json = serde_utils::to_json_str(&index_metadata_inner)?; - for subrequest in request.subrequests { - if let Some(index_id) = subrequest.index_id { - index_ids.push(index_id); - } else if let Some(index_uid) = subrequest.index_uid { - index_uids.push(index_uid); - } else { - let failure = IndexMetadataFailure { - index_id: subrequest.index_id, - index_uid: subrequest.index_uid, - reason: IndexMetadataFailureReason::Internal as i32, - }; - failures.push(failure); - } - } - let pg_indexes: Vec = sqlx::query_as::<_, PgIndex>(INDEXES_METADATA_QUERY) - .bind(&index_ids) - .bind(&index_uids) - .fetch_all(&self.connection_pool) - .await?; + // table_name is a &'static str from ParquetSplitKind, safe for SQL interpolation. + let publish_query = format!( + r#" + WITH publish AS ( + UPDATE {table_name} + SET + split_state = 'Published', + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + WHERE + index_uid = $1 + AND split_id = ANY($3) + AND split_state = 'Staged' + RETURNING split_id + ), + mark_for_deletion AS ( + UPDATE {table_name} + SET + split_state = 'MarkedForDeletion', + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + WHERE + index_uid = $1 + AND split_id = ANY($4) + AND split_state = 'Published' + RETURNING split_id + ), + updated_index_metadata AS ( + UPDATE indexes + SET + index_metadata_json = $2 + WHERE + index_uid = $1 + ) + SELECT + (SELECT COUNT(*) FROM publish) as published_count, + (SELECT COUNT(*) FROM mark_for_deletion) as marked_count + "#, + ); - let indexes_metadata: Vec = pg_indexes - .iter() - .map(|pg_index| pg_index.index_metadata()) - .collect::>()?; + let (published_count, marked_count): (i64, i64) = sqlx::query_as(&publish_query) + .bind(&index_uid_inner) + .bind(index_metadata_json) + .bind(&staged_split_ids) + .bind(&replaced_split_ids) + .fetch_one(tx_ref.as_mut()) + .await + .map_err(|sqlx_error| { + convert_sqlx_err(&index_uid_inner.index_id, sqlx_error) + })?; - if pg_indexes.len() + failures.len() < num_subrequests { - for index_id in index_ids { - if pg_indexes - .iter() - .all(|pg_index| pg_index.index_id != index_id) - { - let failure = IndexMetadataFailure { - index_id: Some(index_id), - index_uid: None, - reason: IndexMetadataFailureReason::NotFound as i32, + if published_count as usize != staged_split_ids.len() { + let entity = EntityKind::Splits { + split_ids: staged_split_ids.clone(), }; - failures.push(failure); + let message = format!( + "expected to publish {} splits, but only {} were in Staged state", + staged_split_ids.len(), + published_count + ); + return Err(MetastoreError::FailedPrecondition { entity, message }); } - } - for index_uid in index_uids { - if pg_indexes - .iter() - .all(|pg_index| pg_index.index_uid != index_uid) - { - let failure = IndexMetadataFailure { - index_id: None, - index_uid: Some(index_uid), - reason: IndexMetadataFailureReason::NotFound as i32, + + if marked_count as usize != replaced_split_ids.len() { + let entity = EntityKind::Splits { + split_ids: replaced_split_ids.clone(), }; - failures.push(failure); + let message = format!( + "expected to mark {} splits for deletion, but only {} were in Published \ + state", + replaced_split_ids.len(), + marked_count + ); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + + info!( + index_uid = %index_uid_inner, + published_count, + marked_count, + "published {label} splits successfully" + ); + Ok(EmptyResponse {}) + } + .await; + match &op_result { + Ok(_) => { + debug!("committing transaction"); + tx.commit().await?; + } + Err(error) => { + rate_limited_error!(limit_per_min = 60, error=%error, "failed to publish {} splits, rolling transaction back", label); + tx.rollback().await?; } } + op_result } - let response = - IndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata, failures).await?; - Ok(response) } - #[instrument(skip(self))] - async fn list_indexes_metadata( + /// Shared implementation for listing parquet splits. + async fn list_parquet_splits_impl( &self, - request: ListIndexesMetadataRequest, - ) -> MetastoreResult { - let sql = - build_index_id_patterns_sql_query(&request.index_id_patterns).map_err(|error| { - MetastoreError::Internal { - message: "failed to build `list_indexes_metadata` SQL query".to_string(), - cause: error.to_string(), - } - })?; - let pg_indexes = sqlx::query_as::<_, PgIndex>(&sql) - .fetch_all(&self.connection_pool) - .await?; - let indexes_metadata: Vec = pg_indexes - .into_iter() - .map(|pg_index| pg_index.index_metadata()) - .collect::>()?; - let response = - ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata).await?; - Ok(response) - } + kind: ParquetSplitKind, + query: ListParquetSplitsQuery, + ) -> MetastoreResult> { + let table_name = kind.table_name(); - #[instrument(skip_all, fields(index_id=%request.index_uid()))] - async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let delete_result = sqlx::query("DELETE FROM indexes WHERE index_uid = $1") - .bind(&index_uid) - .execute(&self.connection_pool) - .await?; - // FIXME: This is not idempotent. - if delete_result.rows_affected() == 0 { - return Err(MetastoreError::NotFound(EntityKind::Index { - index_id: index_uid.index_id, - })); - } - info!(index_id = index_uid.index_id, "deleted index successfully"); - Ok(EmptyResponse {}) - } + let mut sql = format!( + r#" + SELECT + split_id, + split_state, + index_uid, + time_range_start, + time_range_end, + metric_names, + tag_service, + tag_env, + tag_datacenter, + tag_region, + tag_host, + high_cardinality_tag_keys, + num_rows, + size_bytes, + split_metadata_json, + EXTRACT(EPOCH FROM update_timestamp)::bigint as update_timestamp + FROM {table_name} + WHERE index_uid = $1 + "#, + ); - #[instrument(skip_all, fields(split_ids))] - async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let splits_metadata = request.deserialize_splits_metadata()?; + let mut param_idx = 2; - if splits_metadata.is_empty() { - return Ok(Default::default()); + if !query.split_states.is_empty() { + sql.push_str(&format!(" AND split_state = ANY(${}::text[])", param_idx)); + param_idx += 1; + } + if query.time_range_start.is_some() { + sql.push_str(&format!(" AND time_range_end >= ${}", param_idx)); + param_idx += 1; + } + if query.time_range_end.is_some() { + sql.push_str(&format!(" AND time_range_start <= ${}", param_idx)); + param_idx += 1; + } + if !query.metric_names.is_empty() { + sql.push_str(&format!(" AND metric_names && ${}::text[]", param_idx)); + param_idx += 1; + } + if query.tag_service.is_some() { + sql.push_str(&format!(" AND ${} = ANY(tag_service)", param_idx)); + param_idx += 1; + } + if query.tag_env.is_some() { + sql.push_str(&format!(" AND ${} = ANY(tag_env)", param_idx)); + param_idx += 1; + } + if query.tag_datacenter.is_some() { + sql.push_str(&format!(" AND ${} = ANY(tag_datacenter)", param_idx)); + param_idx += 1; + } + if query.tag_region.is_some() { + sql.push_str(&format!(" AND ${} = ANY(tag_region)", param_idx)); + param_idx += 1; + } + if query.tag_host.is_some() { + sql.push_str(&format!(" AND ${} = ANY(tag_host)", param_idx)); + param_idx += 1; } - let mut split_ids = Vec::with_capacity(splits_metadata.len()); - let mut time_range_start_list = Vec::with_capacity(splits_metadata.len()); - let mut time_range_end_list = Vec::with_capacity(splits_metadata.len()); - let mut tags_list = Vec::with_capacity(splits_metadata.len()); - let mut splits_metadata_json = Vec::with_capacity(splits_metadata.len()); - let mut delete_opstamps = Vec::with_capacity(splits_metadata.len()); - let mut maturity_timestamps = Vec::with_capacity(splits_metadata.len()); - let mut node_ids = Vec::with_capacity(splits_metadata.len()); - for split_metadata in splits_metadata { - let split_metadata_json = serde_utils::to_json_str(&split_metadata)?; - splits_metadata_json.push(split_metadata_json); + sql.push_str(" ORDER BY time_range_start ASC"); - let time_range_start = split_metadata - .time_range - .as_ref() - .map(|range| *range.start()); - time_range_start_list.push(time_range_start); - maturity_timestamps.push(split_maturity_timestamp(&split_metadata)); + if query.limit.is_some() { + sql.push_str(&format!(" LIMIT ${}", param_idx)); + } - let time_range_end = split_metadata.time_range.map(|range| *range.end()); - time_range_end_list.push(time_range_end); + let mut query_builder = sqlx::query_as::< + _, + ( + String, // split_id + String, // split_state + String, // index_uid + i64, // time_range_start + i64, // time_range_end + Vec, // metric_names + Option>, // tag_service + Option>, // tag_env + Option>, // tag_datacenter + Option>, // tag_region + Option>, // tag_host + Vec, // high_cardinality_tag_keys + i64, // num_rows + i64, // size_bytes + String, // split_metadata_json + i64, // update_timestamp + ), + >(&sql); - let tags: Vec = split_metadata.tags.into_iter().collect(); - tags_list.push(sqlx::types::Json(tags)); - split_ids.push(split_metadata.split_id); - delete_opstamps.push(split_metadata.delete_opstamp as i64); - node_ids.push(split_metadata.node_id); + query_builder = query_builder.bind(query.index_uid.to_string()); + + if !query.split_states.is_empty() { + query_builder = query_builder.bind(&query.split_states); + } + if let Some(start) = query.time_range_start { + query_builder = query_builder.bind(start); + } + if let Some(end) = query.time_range_end { + query_builder = query_builder.bind(end); + } + if !query.metric_names.is_empty() { + query_builder = query_builder.bind(&query.metric_names); + } + if let Some(ref service) = query.tag_service { + query_builder = query_builder.bind(service); + } + if let Some(ref env) = query.tag_env { + query_builder = query_builder.bind(env); + } + if let Some(ref dc) = query.tag_datacenter { + query_builder = query_builder.bind(dc); + } + if let Some(ref region) = query.tag_region { + query_builder = query_builder.bind(region); + } + if let Some(ref host) = query.tag_host { + query_builder = query_builder.bind(host); + } + if let Some(limit) = query.limit { + query_builder = query_builder.bind(limit as i64); } - tracing::Span::current().record("split_ids", format!("{split_ids:?}")); - // TODO: Remove transaction. - run_with_tx!(self.connection_pool, tx, "stage splits", { - let upserted_split_ids: Vec = sqlx::query_scalar(r#" - INSERT INTO splits - (split_id, time_range_start, time_range_end, tags, split_metadata_json, delete_opstamp, maturity_timestamp, split_state, index_uid, node_id) - SELECT - split_id, - time_range_start, - time_range_end, - ARRAY(SELECT json_array_elements_text(tags_json::json)) as tags, - split_metadata_json, - delete_opstamp, - to_timestamp(maturity_timestamp), - $9 as split_state, - $10 as index_uid, - node_id - FROM - UNNEST($1, $2, $3, $4, $5, $6, $7, $8) - AS staged_splits (split_id, time_range_start, time_range_end, tags_json, split_metadata_json, delete_opstamp, maturity_timestamp, node_id) - ON CONFLICT(index_uid, split_id) DO UPDATE - SET - time_range_start = excluded.time_range_start, - time_range_end = excluded.time_range_end, - tags = excluded.tags, - split_metadata_json = excluded.split_metadata_json, - delete_opstamp = excluded.delete_opstamp, - maturity_timestamp = excluded.maturity_timestamp, - node_id = excluded.node_id, - update_timestamp = CURRENT_TIMESTAMP, - create_timestamp = CURRENT_TIMESTAMP - WHERE splits.split_id = excluded.split_id AND splits.split_state = 'Staged' - RETURNING split_id; - "#) - .bind(&split_ids) - .bind(time_range_start_list) - .bind(time_range_end_list) - .bind(tags_list) - .bind(splits_metadata_json) - .bind(delete_opstamps) - .bind(maturity_timestamps) - .bind(&node_ids) - .bind(SplitState::Staged.as_str()) - .bind(&index_uid) - .fetch_all(tx.as_mut()) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + let rows = query_builder + .fetch_all(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(&query.index_uid.index_id, sqlx_error))?; - if upserted_split_ids.len() != split_ids.len() { - let failed_split_ids: Vec = split_ids - .into_iter() - .filter(|split_id| !upserted_split_ids.contains(split_id)) - .collect(); - let entity = EntityKind::Splits { - split_ids: failed_split_ids, + let splits: Vec = rows + .into_iter() + .filter_map(|row| { + let pg_split = PgParquetSplit { + split_id: row.0, + split_state: row.1, + index_uid: row.2, + time_range_start: row.3, + time_range_end: row.4, + metric_names: row.5, + tag_service: row.6, + tag_env: row.7, + tag_datacenter: row.8, + tag_region: row.9, + tag_host: row.10, + high_cardinality_tag_keys: row.11, + num_rows: row.12, + size_bytes: row.13, + split_metadata_json: row.14, + update_timestamp: row.15, }; - let message = "splits are not staged".to_string(); - return Err(MetastoreError::FailedPrecondition { entity, message }); - } - info!( - %index_uid, - "staged `{}` splits successfully", split_ids.len() - ); - Ok(EmptyResponse {}) - }) + + let state = pg_split.split_state().unwrap_or(ParquetSplitState::Staged); + let metadata = pg_split.to_metadata().ok()?; + + Some(ParquetSplitRecord { + state, + update_timestamp: row.15, + metadata, + }) + }) + .collect(); + + Ok(splits) } - #[instrument(skip(self))] - async fn publish_splits( + /// Shared implementation for marking parquet splits for deletion. + async fn mark_parquet_splits_for_deletion_impl( &self, - request: PublishSplitsRequest, + kind: ParquetSplitKind, + index_uid: &IndexUid, + split_ids: &[String], ) -> MetastoreResult { - let checkpoint_delta_opt: Option = - request.deserialize_index_checkpoint()?; - let index_uid: IndexUid = request.index_uid().clone(); - let staged_split_ids = request.staged_split_ids; - let replaced_split_ids = request.replaced_split_ids; + if split_ids.is_empty() { + return Ok(EmptyResponse {}); + } - run_with_tx!(self.connection_pool, tx, "publish splits", { - let mut index_metadata = index_metadata(tx, &index_uid.index_id, true).await?; - if index_metadata.index_uid != index_uid { - return Err(MetastoreError::NotFound(EntityKind::Index { - index_id: index_uid.index_id, - })); - } - if let Some(checkpoint_delta) = checkpoint_delta_opt { - let source_id = checkpoint_delta.source_id.clone(); - let source = index_metadata.sources.get(&source_id).ok_or_else(|| { - MetastoreError::NotFound(EntityKind::Source { - index_id: index_uid.index_id.to_string(), - source_id: source_id.to_string(), - }) - })?; + let table_name = kind.table_name(); + let label = kind.label(); - if use_shard_api(&source.source_params) { - let publish_token = request.publish_token_opt.ok_or_else(|| { - let message = format!( - "publish token is required for publishing splits for source \ - `{source_id}`" - ); - MetastoreError::InvalidArgument { message } - })?; - try_apply_delta_v2( - tx, - &index_uid, - &source_id, - checkpoint_delta.source_delta, - publish_token, - ) - .await?; - } else { - index_metadata - .checkpoint - .try_apply_delta(checkpoint_delta) - .map_err(|error| { - let entity = EntityKind::CheckpointDelta { - index_id: index_uid.index_id.to_string(), - source_id, - }; - let message = error.to_string(); - MetastoreError::FailedPrecondition { entity, message } - })?; - } - } - let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; + info!( + index_uid = %index_uid, + split_ids = ?split_ids, + "marking {label} splits for deletion" + ); - const PUBLISH_SPLITS_QUERY: &str = r#" - -- Select the splits to update, regardless of their state. - -- The left join make it possible to identify the splits that do not exist. - WITH input_splits AS ( - SELECT input_splits.split_id, input_splits.expected_split_state, splits.actual_split_state - FROM ( - SELECT split_id, 'Staged' AS expected_split_state - FROM UNNEST($3) AS staged_splits(split_id) - UNION - SELECT split_id, 'Published' AS expected_split_state - FROM UNNEST($4) AS published_splits(split_id) - ) input_splits - LEFT JOIN ( - SELECT split_id, split_state AS actual_split_state - FROM splits - WHERE - index_uid = $1 - AND (split_id = ANY($3) OR split_id = ANY($4)) - FOR UPDATE - ) AS splits - USING (split_id) - ), - -- Update the index metadata with the new checkpoint. - updated_index_metadata AS ( - UPDATE indexes - SET - index_metadata_json = $2 - WHERE - index_uid = $1 - AND NOT EXISTS ( - SELECT 1 - FROM input_splits - WHERE - actual_split_state != expected_split_state - ) - ), - -- Publish the staged splits and mark the published splits for deletion. - updated_splits AS ( - UPDATE splits - SET - split_state = CASE split_state - WHEN 'Staged' THEN 'Published' - ELSE 'MarkedForDeletion' - END, - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), - publish_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - FROM input_splits - WHERE - splits.index_uid = $1 - AND splits.split_id = input_splits.split_id - AND NOT EXISTS ( - SELECT 1 - FROM input_splits - WHERE - actual_split_state != expected_split_state - ) - ) - -- Report the outcome of the update query. - SELECT - COUNT(1) FILTER (WHERE actual_split_state = 'Staged' AND expected_split_state = 'Staged'), - COUNT(1) FILTER (WHERE actual_split_state = 'Published' AND expected_split_state = 'Published'), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state IS NULL), ARRAY[]::TEXT[]), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state != 'Staged' AND expected_split_state = 'Staged'), ARRAY[]::TEXT[]), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state != 'Published' AND expected_split_state = 'Published'), ARRAY[]::TEXT[]) - FROM input_splits - "#; - let ( - num_published_splits, - num_marked_splits, - not_found_split_ids, - not_staged_split_ids, - not_marked_split_ids, - ): (i64, i64, Vec, Vec, Vec) = - sqlx::query_as(PUBLISH_SPLITS_QUERY) - .bind(&index_uid) - .bind(index_metadata_json) - .bind(staged_split_ids) - .bind(replaced_split_ids) - .fetch_one(tx.as_mut()) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + let mark_query = format!( + r#" + UPDATE {table_name} + SET + split_state = 'MarkedForDeletion', + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + WHERE + index_uid = $1 + AND split_id = ANY($2) + AND split_state IN ('Staged', 'Published') + RETURNING split_id + "#, + ); - if !not_found_split_ids.is_empty() { - return Err(MetastoreError::NotFound(EntityKind::Splits { - split_ids: not_found_split_ids, - })); - } - if !not_staged_split_ids.is_empty() { - let entity = EntityKind::Splits { - split_ids: not_staged_split_ids, - }; - let message = "splits are not staged".to_string(); - return Err(MetastoreError::FailedPrecondition { entity, message }); - } - if !not_marked_split_ids.is_empty() { - let entity = EntityKind::Splits { - split_ids: not_marked_split_ids, - }; - let message = "splits are not marked for deletion".to_string(); - return Err(MetastoreError::FailedPrecondition { entity, message }); - } - info!( - %index_uid, - "published {num_published_splits} splits and marked {num_marked_splits} for deletion successfully" - ); - Ok(EmptyResponse {}) - }) + let marked_split_ids: Vec = sqlx::query_scalar(&mark_query) + .bind(index_uid) + .bind(split_ids) + .fetch_all(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + + info!( + index_uid = %index_uid, + marked_count = marked_split_ids.len(), + "marked {label} splits for deletion" + ); + Ok(EmptyResponse {}) } - #[instrument(skip(self))] - async fn list_splits( + /// Shared implementation for deleting parquet splits. + async fn delete_parquet_splits_impl( &self, - request: ListSplitsRequest, - ) -> MetastoreResult> { - let list_splits_query = request.deserialize_list_splits_query()?; - let mut sql_query_builder = Query::select(); - sql_query_builder.column(Asterisk).from(Splits::Table); - append_query_filters_and_order_by(&mut sql_query_builder, &list_splits_query); + kind: ParquetSplitKind, + index_uid: &IndexUid, + split_ids: &[String], + ) -> MetastoreResult { + if split_ids.is_empty() { + return Ok(EmptyResponse {}); + } - let (sql_query, values) = sql_query_builder.build_sqlx(PostgresQueryBuilder); - let pg_split_stream = SplitStream::new( - self.connection_pool.clone(), - sql_query, - |connection_pool: &TrackedPool, sql_query: &String| { - sqlx::query_as_with::<_, PgSplit, _>(sql_query, values).fetch(connection_pool) - }, + let table_name = kind.table_name(); + let label = kind.label(); + + info!( + index_uid = %index_uid, + split_ids = ?split_ids, + "deleting {label} splits" ); - let split_stream = - pg_split_stream - .chunks(STREAM_SPLITS_CHUNK_SIZE) - .map(|pg_splits_results| { - let mut splits = Vec::with_capacity(pg_splits_results.len()); - for pg_split_result in pg_splits_results { - let pg_split = match pg_split_result { - Ok(pg_split) => pg_split, - Err(error) => { - return Err(MetastoreError::Internal { - message: "failed to fetch splits".to_string(), - cause: error.to_string(), - }); - } - }; - let split: Split = match pg_split.try_into() { - Ok(split) => split, - Err(error) => { - return Err(MetastoreError::Internal { - message: "failed to convert `PgSplit` to `Split`".to_string(), - cause: error.to_string(), - }); - } - }; - splits.push(split); - } - ListSplitsResponse::try_from_splits(splits) - }); - let service_stream = ServiceStream::new(Box::pin(split_stream)); - Ok(service_stream) - } - async fn list_index_stats( - &self, - request: ListIndexStatsRequest, - ) -> MetastoreResult { - let index_pattern_sql = build_index_id_patterns_sql_query(&request.index_id_patterns) - .map_err(|error| MetastoreError::Internal { - message: "failed to build `list_index_stats` SQL query".to_string(), - cause: error.to_string(), - })?; - let sql = format!( - "SELECT - i.index_uid, - s.split_state, - COUNT(s.split_state) AS num_splits, - COALESCE(SUM(s.split_size_bytes)::BIGINT, 0) AS total_size_bytes - FROM ({index_pattern_sql}) i - LEFT JOIN splits s ON s.index_uid = i.index_uid - GROUP BY i.index_uid, s.split_state" + let delete_query = format!( + r#" + DELETE FROM {table_name} + WHERE + index_uid = $1 + AND split_id = ANY($2) + AND split_state = 'MarkedForDeletion' + RETURNING split_id + "#, ); - let rows: Vec<(String, Option, i64, i64)> = sqlx::query_as(&sql) + let deleted_split_ids: Vec = sqlx::query_scalar(&delete_query) + .bind(index_uid) + .bind(split_ids) .fetch_all(&self.connection_pool) - .await?; + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; - let mut index_stats = HashMap::new(); - for (index_uid_str, split_state, num_splits, total_size_bytes) in rows { - let Ok(index_uid) = IndexUid::from_str(&index_uid_str) else { - return Err(MetastoreError::Internal { - message: "failed to parse index_uid".to_string(), - cause: index_uid_str.to_string(), - }); - }; - let stats = index_stats - .entry(index_uid_str) - .or_insert_with(|| IndexStats { - index_uid: Some(index_uid), - staged: Some(SplitStats::default()), - published: Some(SplitStats::default()), - marked_for_deletion: Some(SplitStats::default()), - }); - let num_splits = num_splits as u64; - let total_size_bytes = total_size_bytes as u64; - match split_state.as_deref() { - Some("Staged") => { - stats.staged = Some(SplitStats { - num_splits, - total_size_bytes, - }); - } - Some("Published") => { - stats.published = Some(SplitStats { - num_splits, - total_size_bytes, - }); - } - Some("MarkedForDeletion") => { - stats.marked_for_deletion = Some(SplitStats { - num_splits, - total_size_bytes, - }); - } - None => {} // if an index has no splits, we can keep the defaults - Some(split_state) => { - return Err(MetastoreError::Internal { - message: "invalid split state".to_string(), - cause: split_state.to_string(), - }); - } + if deleted_split_ids.len() != split_ids.len() { + let not_deleted: Vec = split_ids + .iter() + .filter(|id| !deleted_split_ids.contains(id)) + .cloned() + .collect(); + + if !not_deleted.is_empty() { + warn!( + index_uid = %index_uid, + not_deleted = ?not_deleted, + "some {label} splits were not in MarkedForDeletion state" + ); + let entity = EntityKind::Splits { + split_ids: not_deleted, + }; + let message = "splits are not marked for deletion".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); } } - Ok(ListIndexStatsResponse { - index_stats: index_stats.into_values().collect(), - }) + info!( + index_uid = %index_uid, + deleted_count = deleted_split_ids.len(), + "deleted {label} splits successfully" + ); + Ok(EmptyResponse {}) + } +} + +#[async_trait] +impl MetastoreService for PostgresqlMetastore { + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.connection_pool.acquire().await?; + Ok(()) + } + + fn endpoints(&self) -> Vec { + vec![self.uri.clone()] } + // Index API: + // - `create_index` + // - `update_index` + // - `index_metadata` + // - `indexes_metadata` + // - `list_indexes_metadata` + #[instrument(skip(self))] - async fn mark_splits_for_deletion( + async fn create_index( &self, - request: MarkSplitsForDeletionRequest, - ) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let split_ids = request.split_ids; - const MARK_SPLITS_FOR_DELETION_QUERY: &str = r#" - -- Select the splits to update, regardless of their state. - -- The left join make it possible to identify the splits that do not exist. - WITH input_splits AS ( - SELECT input_splits.split_id, splits.split_state - FROM UNNEST($2) AS input_splits(split_id) - LEFT JOIN ( - SELECT split_id, split_state - FROM splits - WHERE - index_uid = $1 - AND split_id = ANY($2) - FOR UPDATE - ) AS splits - USING (split_id) - ), - -- Mark the staged and published splits for deletion. - marked_splits AS ( - UPDATE splits - SET - split_state = 'MarkedForDeletion', - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - FROM input_splits - WHERE - splits.index_uid = $1 - AND splits.split_id = input_splits.split_id - AND splits.split_state IN ('Staged', 'Published') - ) - -- Report the outcome of the update query. - SELECT - COUNT(split_state), - COUNT(1) FILTER (WHERE split_state IN ('Staged', 'Published')), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) - FROM input_splits - "#; - let (num_found_splits, num_marked_splits, not_found_split_ids): (i64, i64, Vec) = - sqlx::query_as(MARK_SPLITS_FOR_DELETION_QUERY) - .bind(&index_uid) - .bind(split_ids.clone()) - .fetch_one(&self.connection_pool) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; - - if num_found_splits == 0 - && index_opt(&self.connection_pool, &index_uid.index_id, false) - .await? - .is_none() - { - return Err(MetastoreError::NotFound(EntityKind::Index { - index_id: index_uid.index_id, - })); - } - info!( - %index_uid, - "Marked {} splits for deletion, among which {} were newly marked.", - split_ids.len() - not_found_split_ids.len(), - num_marked_splits - ); - if !not_found_split_ids.is_empty() { - warn!( - %index_uid, - split_ids=?PrettySample::new(¬_found_split_ids, 5), - "{} splits were not found and could not be marked for deletion.", - not_found_split_ids.len() - ); - } - Ok(EmptyResponse {}) - } + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_config = request.deserialize_index_config()?; + let mut index_metadata = IndexMetadata::new(index_config); - #[instrument(skip(self))] - async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let split_ids = request.split_ids; - const DELETE_SPLITS_QUERY: &str = r#" - -- Select the splits to delete, regardless of their state. - -- The left join make it possible to identify the splits that do not exist. - WITH input_splits AS ( - SELECT input_splits.split_id, splits.split_state - FROM UNNEST($2) AS input_splits(split_id) - LEFT JOIN ( - SELECT split_id, split_state - FROM splits - WHERE - index_uid = $1 - AND split_id = ANY($2) - FOR UPDATE - ) AS splits - USING (split_id) - ), - -- Delete the splits if and only if all the splits are marked for deletion. - deleted_splits AS ( - DELETE FROM splits - USING input_splits - WHERE - splits.index_uid = $1 - AND splits.split_id = input_splits.split_id - AND NOT EXISTS ( - SELECT 1 - FROM input_splits - WHERE - split_state IN ('Staged', 'Published') - ) - ) - -- Report the outcome of the delete query. - SELECT - COUNT(split_state), - COUNT(1) FILTER (WHERE split_state = 'MarkedForDeletion'), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IN ('Staged', 'Published')), ARRAY[]::TEXT[]), - COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) - FROM input_splits - "#; - let (num_found_splits, num_deleted_splits, not_deletable_split_ids, not_found_split_ids): ( - i64, - i64, - Vec, - Vec, - ) = sqlx::query_as(DELETE_SPLITS_QUERY) - .bind(&index_uid) - .bind(split_ids) - .fetch_one(&self.connection_pool) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + let source_configs = request.deserialize_source_configs()?; - if num_found_splits == 0 - && index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) - .await? - .is_none() - { - return Err(MetastoreError::NotFound(EntityKind::Index { - index_id: index_uid.index_id, - })); - } - if !not_deletable_split_ids.is_empty() { - let message = format!( - "splits `{}` are not deletable", - not_deletable_split_ids.join(", ") - ); - let entity = EntityKind::Splits { - split_ids: not_deletable_split_ids, - }; - return Err(MetastoreError::FailedPrecondition { entity, message }); + for source_config in source_configs { + index_metadata.add_source(source_config)?; } - info!(%index_uid, "deleted {} splits from index", num_deleted_splits); + let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; - if !not_found_split_ids.is_empty() { - warn!( - %index_uid, - split_ids=?PrettySample::new(¬_found_split_ids, 5), - "{} splits were not found and could not be deleted.", - not_found_split_ids.len() - ); - } - Ok(EmptyResponse {}) - } + sqlx::query( + "INSERT INTO indexes (index_uid, index_id, index_metadata_json) VALUES ($1, $2, $3)", + ) + .bind(index_metadata.index_uid.to_string()) + .bind(&index_metadata.index_uid.index_id) + .bind(&index_metadata_json) + .execute(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(index_metadata.index_id(), sqlx_error))?; - #[instrument(skip(self))] - async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { - let source_config = request.deserialize_source_config()?; - let index_uid: IndexUid = request.index_uid().clone(); - run_with_tx!(self.connection_pool, tx, "add source", { - mutate_index_metadata::(tx, index_uid, |index_metadata| { - index_metadata.add_source(source_config)?; - Ok(MutationOccurred::Yes(())) - }) - .await?; - Ok(()) - })?; - Ok(EmptyResponse {}) + let response = CreateIndexResponse { + index_uid: index_metadata.index_uid.into(), + index_metadata_json, + }; + Ok(response) } - #[instrument(skip(self))] - async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { - let source_config = request.deserialize_source_config()?; + async fn update_index( + &self, + request: UpdateIndexRequest, + ) -> MetastoreResult { + let doc_mapping = request.deserialize_doc_mapping()?; + let indexing_settings = request.deserialize_indexing_settings()?; + let ingest_settings = request.deserialize_ingest_settings()?; + let search_settings = request.deserialize_search_settings()?; + let retention_policy_opt = request.deserialize_retention_policy()?; + let index_uid: IndexUid = request.index_uid().clone(); - run_with_tx!(self.connection_pool, tx, "update source", { + let updated_index_metadata = run_with_tx!(self.connection_pool, tx, "update index", { mutate_index_metadata::(tx, index_uid, |index_metadata| { - let mutation_occurred = index_metadata.update_source(source_config)?; + let mutation_occurred = index_metadata.update_index_config( + doc_mapping, + indexing_settings, + ingest_settings, + search_settings, + retention_policy_opt, + )?; Ok(MutationOccurred::from(mutation_occurred)) }) - .await?; - Ok(()) + .await })?; - Ok(EmptyResponse {}) + IndexMetadataResponse::try_from_index_metadata(&updated_index_metadata) } #[instrument(skip(self))] - async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - run_with_tx!(self.connection_pool, tx, "toggle source", { - mutate_index_metadata(tx, index_uid, |index_metadata| { - if index_metadata.toggle_source(&request.source_id, request.enable)? { - Ok::<_, MetastoreError>(MutationOccurred::Yes(())) - } else { - Ok::<_, MetastoreError>(MutationOccurred::No(())) - } - }) - .await?; - Ok(()) - })?; - Ok(EmptyResponse {}) + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let pg_index_opt = if let Some(index_uid) = &request.index_uid { + index_opt_for_uid(&self.connection_pool, index_uid.clone(), false).await? + } else if let Some(index_id) = &request.index_id { + index_opt(&self.connection_pool, index_id, false).await? + } else { + let message = "invalid request: neither `index_id` nor `index_uid` is set".to_string(); + return Err(MetastoreError::Internal { + message, + cause: "".to_string(), + }); + }; + let index_metadata = pg_index_opt + .ok_or(MetastoreError::NotFound(EntityKind::Index { + index_id: request + .into_index_id() + .expect("`index_id` or `index_uid` should be set"), + }))? + .index_metadata()?; + let response = IndexMetadataResponse::try_from_index_metadata(&index_metadata)?; + Ok(response) } #[instrument(skip(self))] - async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let source_id = request.source_id.clone(); - run_with_tx!(self.connection_pool, tx, "delete source", { - mutate_index_metadata(tx, index_uid.clone(), |index_metadata| { - index_metadata.delete_source(&source_id)?; - Ok::<_, MetastoreError>(MutationOccurred::Yes(())) - }) - .await?; - sqlx::query( - r#" - DELETE FROM shards - WHERE - index_uid = $1 - AND source_id = $2 - "#, - ) - .bind(&index_uid) - .bind(source_id) - .execute(tx.as_mut()) - .await?; - Ok(()) - })?; - Ok(EmptyResponse {}) - } - - #[instrument(skip(self))] - async fn reset_source_checkpoint( + async fn indexes_metadata( &self, - request: ResetSourceCheckpointRequest, - ) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - run_with_tx!(self.connection_pool, tx, "reset source checkpoint", { - mutate_index_metadata(tx, index_uid, |index_metadata| { - if index_metadata.checkpoint.reset_source(&request.source_id) { - Ok::<_, MetastoreError>(MutationOccurred::Yes(())) - } else { - Ok::<_, MetastoreError>(MutationOccurred::No(())) - } - }) - .await?; - Ok(()) - })?; - Ok(EmptyResponse {}) - } + request: IndexesMetadataRequest, + ) -> MetastoreResult { + const INDEXES_METADATA_QUERY: &str = include_str!("queries/indexes_metadata.sql"); - /// Retrieves the last delete opstamp for a given `index_id`. - #[instrument(skip(self))] - async fn last_delete_opstamp( - &self, - request: LastDeleteOpstampRequest, - ) -> MetastoreResult { - let max_opstamp: i64 = sqlx::query_scalar( - r#" - SELECT COALESCE(MAX(opstamp), 0) - FROM delete_tasks - WHERE index_uid = $1 - "#, - ) - .bind(request.index_uid()) - .fetch_one(&self.connection_pool) - .await - .map_err(|error| MetastoreError::Db { - message: error.to_string(), - })?; + let num_subrequests = request.subrequests.len(); - Ok(LastDeleteOpstampResponse::new(max_opstamp as u64)) - } + if num_subrequests == 0 { + return Ok(Default::default()); + } + let mut index_ids: Vec = Vec::new(); + let mut index_uids: Vec = Vec::with_capacity(num_subrequests); + let mut failures: Vec = Vec::new(); - /// Creates a delete task from a delete query. - #[instrument(skip(self))] - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { - let delete_query_json = serde_utils::to_json_str(&delete_query)?; - let (create_timestamp, opstamp): (sqlx::types::time::PrimitiveDateTime, i64) = - sqlx::query_as( - r#" - INSERT INTO delete_tasks (index_uid, delete_query_json) VALUES ($1, $2) - RETURNING create_timestamp, opstamp - "#, - ) - .bind(delete_query.index_uid().to_string()) - .bind(&delete_query_json) - .fetch_one(&self.connection_pool) - .await - .map_err(|error| convert_sqlx_err(&delete_query.index_uid().index_id, error))?; + for subrequest in request.subrequests { + if let Some(index_id) = subrequest.index_id { + index_ids.push(index_id); + } else if let Some(index_uid) = subrequest.index_uid { + index_uids.push(index_uid); + } else { + let failure = IndexMetadataFailure { + index_id: subrequest.index_id, + index_uid: subrequest.index_uid, + reason: IndexMetadataFailureReason::Internal as i32, + }; + failures.push(failure); + } + } + let pg_indexes: Vec = sqlx::query_as::<_, PgIndex>(INDEXES_METADATA_QUERY) + .bind(&index_ids) + .bind(&index_uids) + .fetch_all(&self.connection_pool) + .await?; - Ok(DeleteTask { - create_timestamp: create_timestamp.assume_utc().unix_timestamp(), - opstamp: opstamp as u64, - delete_query: Some(delete_query), - }) + let indexes_metadata: Vec = pg_indexes + .iter() + .map(|pg_index| pg_index.index_metadata()) + .collect::>()?; + + if pg_indexes.len() + failures.len() < num_subrequests { + for index_id in index_ids { + if pg_indexes + .iter() + .all(|pg_index| pg_index.index_id != index_id) + { + let failure = IndexMetadataFailure { + index_id: Some(index_id), + index_uid: None, + reason: IndexMetadataFailureReason::NotFound as i32, + }; + failures.push(failure); + } + } + for index_uid in index_uids { + if pg_indexes + .iter() + .all(|pg_index| pg_index.index_uid != index_uid) + { + let failure = IndexMetadataFailure { + index_id: None, + index_uid: Some(index_uid), + reason: IndexMetadataFailureReason::NotFound as i32, + }; + failures.push(failure); + } + } + } + let response = + IndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata, failures).await?; + Ok(response) } - /// Update splits delete opstamps. #[instrument(skip(self))] - async fn update_splits_delete_opstamp( + async fn list_indexes_metadata( &self, - request: UpdateSplitsDeleteOpstampRequest, - ) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let split_ids = request.split_ids; - if split_ids.is_empty() { - return Ok(UpdateSplitsDeleteOpstampResponse {}); - } - let update_result = sqlx::query( - r#" - UPDATE splits - SET - delete_opstamp = $1, - -- The values we compare with are *before* the modification: - update_timestamp = CASE - WHEN delete_opstamp != $1 THEN (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - ELSE update_timestamp - END - WHERE - index_uid = $2 - AND split_id = ANY($3) - "#, - ) - .bind(request.delete_opstamp as i64) - .bind(&index_uid) - .bind(split_ids) - .execute(&self.connection_pool) - .await?; + request: ListIndexesMetadataRequest, + ) -> MetastoreResult { + let sql = + build_index_id_patterns_sql_query(&request.index_id_patterns).map_err(|error| { + MetastoreError::Internal { + message: "failed to build `list_indexes_metadata` SQL query".to_string(), + cause: error.to_string(), + } + })?; + let pg_indexes = sqlx::query_as::<_, PgIndex>(&sql) + .fetch_all(&self.connection_pool) + .await?; + let indexes_metadata: Vec = pg_indexes + .into_iter() + .map(|pg_index| pg_index.index_metadata()) + .collect::>()?; + let response = + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata).await?; + Ok(response) + } - // If no splits were updated, maybe the index does not exist in the first place? - if update_result.rows_affected() == 0 - && index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) - .await? - .is_none() - { + #[instrument(skip_all, fields(index_id=%request.index_uid()))] + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let delete_result = sqlx::query("DELETE FROM indexes WHERE index_uid = $1") + .bind(&index_uid) + .execute(&self.connection_pool) + .await?; + // FIXME: This is not idempotent. + if delete_result.rows_affected() == 0 { return Err(MetastoreError::NotFound(EntityKind::Index { index_id: index_uid.index_id, })); } - Ok(UpdateSplitsDeleteOpstampResponse {}) + info!(index_id = index_uid.index_id, "deleted index successfully"); + Ok(EmptyResponse {}) } - /// Lists the delete tasks with opstamp > `opstamp_start`. - #[instrument(skip(self))] - async fn list_delete_tasks( - &self, - request: ListDeleteTasksRequest, - ) -> MetastoreResult { + #[instrument(skip_all, fields(split_ids))] + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); - let pg_delete_tasks: Vec = sqlx::query_as::<_, PgDeleteTask>( - r#" - SELECT * FROM delete_tasks - WHERE - index_uid = $1 - AND opstamp > $2 - "#, - ) - .bind(&index_uid) - .bind(request.opstamp_start as i64) - .fetch_all(&self.connection_pool) - .await?; - let delete_tasks: Vec = pg_delete_tasks - .into_iter() - .map(|pg_delete_task| pg_delete_task.try_into()) - .collect::>()?; - Ok(ListDeleteTasksResponse { delete_tasks }) - } + let splits_metadata = request.deserialize_splits_metadata()?; - /// Returns `num_splits` published splits with `split.delete_opstamp` < `delete_opstamp`. - /// Results are ordered by ascending `split.delete_opstamp` and `split.publish_timestamp` - /// values. - #[instrument(skip(self))] - async fn list_stale_splits( - &self, - request: ListStaleSplitsRequest, - ) -> MetastoreResult { - let index_uid: IndexUid = request.index_uid().clone(); - let stale_pg_splits: Vec = sqlx::query_as::<_, PgSplit>( - r#" - SELECT * - FROM splits - WHERE - index_uid = $1 - AND delete_opstamp < $2 - AND split_state = $3 - AND (maturity_timestamp = to_timestamp(0) OR (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') >= maturity_timestamp) - ORDER BY delete_opstamp ASC, publish_timestamp ASC - LIMIT $4 - "#, - ) - .bind(&index_uid) - .bind(request.delete_opstamp as i64) - .bind(SplitState::Published.as_str()) - .bind(request.num_splits as i64) - .fetch_all(&self.connection_pool) - .await?; - - let stale_splits: Vec = stale_pg_splits - .into_iter() - .map(|pg_split| pg_split.try_into()) - .collect::>()?; - let response = ListSplitsResponse::try_from_splits(stale_splits)?; - Ok(response) - } - - // TODO: Issue a single SQL query. - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { - let mut subresponses = Vec::with_capacity(request.subrequests.len()); - - for subrequest in request.subrequests { - let open_shard: Shard = open_or_fetch_shard(&self.connection_pool, &subrequest).await?; - let subresponse = OpenShardSubresponse { - subrequest_id: subrequest.subrequest_id, - open_shard: Some(open_shard), - }; - subresponses.push(subresponse); - } - Ok(OpenShardsResponse { subresponses }) - } - - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult { - const ACQUIRE_SHARDS_QUERY: &str = include_str!("queries/shards/acquire.sql"); - - if request.shard_ids.is_empty() { - return Ok(Default::default()); - } - let pg_shards: Vec = sqlx::query_as(ACQUIRE_SHARDS_QUERY) - .bind(request.index_uid()) - .bind(&request.source_id) - .bind(&request.shard_ids) - .bind(&request.publish_token) - .fetch_all(&self.connection_pool) - .await?; - let acquired_shards = pg_shards - .into_iter() - .map(|pg_shard| pg_shard.into()) - .collect(); - let response = AcquireShardsResponse { acquired_shards }; - Ok(response) - } - - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - if request.subrequests.is_empty() { + if splits_metadata.is_empty() { return Ok(Default::default()); } - let mut sql_query_builder = Query::select(); + let mut split_ids = Vec::with_capacity(splits_metadata.len()); + let mut time_range_start_list = Vec::with_capacity(splits_metadata.len()); + let mut time_range_end_list = Vec::with_capacity(splits_metadata.len()); + let mut tags_list = Vec::with_capacity(splits_metadata.len()); + let mut splits_metadata_json = Vec::with_capacity(splits_metadata.len()); + let mut delete_opstamps = Vec::with_capacity(splits_metadata.len()); + let mut maturity_timestamps = Vec::with_capacity(splits_metadata.len()); + let mut node_ids = Vec::with_capacity(splits_metadata.len()); - for (idx, subrequest) in request.subrequests.iter().enumerate() { - let mut sql_subquery_builder = Query::select(); + for split_metadata in splits_metadata { + let split_metadata_json = serde_utils::to_json_str(&split_metadata)?; + splits_metadata_json.push(split_metadata_json); - sql_subquery_builder - .column(Asterisk) - .from(Shards::Table) - .and_where(Expr::col(Shards::IndexUid).eq(subrequest.index_uid())) - .and_where(Expr::col(Shards::SourceId).eq(&subrequest.source_id)); + let time_range_start = split_metadata + .time_range + .as_ref() + .map(|range| *range.start()); + time_range_start_list.push(time_range_start); + maturity_timestamps.push(split_maturity_timestamp(&split_metadata)); - let shard_state = subrequest.shard_state(); + let time_range_end = split_metadata.time_range.map(|range| *range.end()); + time_range_end_list.push(time_range_end); - if shard_state != ShardState::Unspecified { - let shard_state_str = shard_state.as_json_str_name(); - let shard_state_alias = Alias::new("SHARD_STATE"); - let cast_expr = Func::cast_as(shard_state_str, shard_state_alias); - sql_subquery_builder.and_where(Expr::col(Shards::ShardState).eq(cast_expr)); - } - if idx == 0 { - sql_query_builder = sql_subquery_builder; - } else { - sql_query_builder.union(UnionType::All, sql_subquery_builder); - } + let tags: Vec = split_metadata.tags.into_iter().collect(); + tags_list.push(sqlx::types::Json(tags)); + split_ids.push(split_metadata.split_id); + delete_opstamps.push(split_metadata.delete_opstamp as i64); + node_ids.push(split_metadata.node_id); } - let (sql_query, values) = sql_query_builder.build_sqlx(PostgresQueryBuilder); - - let pg_shards: Vec = sqlx::query_as_with::<_, PgShard, _>(&sql_query, values) - .fetch_all(&self.connection_pool) - .await?; - - let mut per_source_subresponses: HashMap<(IndexUid, SourceId), ListShardsSubresponse> = - request - .subrequests - .into_iter() - .map(|subrequest| { - let index_uid = subrequest.index_uid().clone(); - let source_id = subrequest.source_id.clone(); - ( - (index_uid, source_id), - ListShardsSubresponse { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shards: Vec::new(), - }, - ) - }) - .collect(); + tracing::Span::current().record("split_ids", format!("{split_ids:?}")); - for pg_shard in pg_shards { - let shard: Shard = pg_shard.into(); - let source_key = (shard.index_uid().clone(), shard.source_id.clone()); + // TODO: Remove transaction. + run_with_tx!(self.connection_pool, tx, "stage splits", { + let upserted_split_ids: Vec = sqlx::query_scalar(r#" + INSERT INTO splits + (split_id, time_range_start, time_range_end, tags, split_metadata_json, delete_opstamp, maturity_timestamp, split_state, index_uid, node_id) + SELECT + split_id, + time_range_start, + time_range_end, + ARRAY(SELECT json_array_elements_text(tags_json::json)) as tags, + split_metadata_json, + delete_opstamp, + to_timestamp(maturity_timestamp), + $9 as split_state, + $10 as index_uid, + node_id + FROM + UNNEST($1, $2, $3, $4, $5, $6, $7, $8) + AS staged_splits (split_id, time_range_start, time_range_end, tags_json, split_metadata_json, delete_opstamp, maturity_timestamp, node_id) + ON CONFLICT(index_uid, split_id) DO UPDATE + SET + time_range_start = excluded.time_range_start, + time_range_end = excluded.time_range_end, + tags = excluded.tags, + split_metadata_json = excluded.split_metadata_json, + delete_opstamp = excluded.delete_opstamp, + maturity_timestamp = excluded.maturity_timestamp, + node_id = excluded.node_id, + update_timestamp = CURRENT_TIMESTAMP, + create_timestamp = CURRENT_TIMESTAMP + WHERE splits.split_id = excluded.split_id AND splits.split_state = 'Staged' + RETURNING split_id; + "#) + .bind(&split_ids) + .bind(time_range_start_list) + .bind(time_range_end_list) + .bind(tags_list) + .bind(splits_metadata_json) + .bind(delete_opstamps) + .bind(maturity_timestamps) + .bind(&node_ids) + .bind(ParquetSplitState::Staged.as_str()) + .bind(&index_uid) + .fetch_all(tx.as_mut()) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; - let Some(subresponse) = per_source_subresponses.get_mut(&source_key) else { - warn!( - index_uid=%shard.index_uid(), - source_id=%shard.source_id, - "could not find source in subresponses: this should never happen, please report" - ); - continue; - }; - subresponse.shards.push(shard); - } - let subresponses = per_source_subresponses.into_values().collect(); - let response = ListShardsResponse { subresponses }; - Ok(response) + if upserted_split_ids.len() != split_ids.len() { + let failed_split_ids: Vec = split_ids + .into_iter() + .filter(|split_id| !upserted_split_ids.contains(split_id)) + .collect(); + let entity = EntityKind::Splits { + split_ids: failed_split_ids, + }; + let message = "splits are not staged".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + info!( + %index_uid, + "staged `{}` splits successfully", split_ids.len() + ); + Ok(EmptyResponse {}) + }) } - async fn delete_shards( + #[instrument(skip(self))] + async fn publish_splits( &self, - request: DeleteShardsRequest, - ) -> MetastoreResult { - const DELETE_SHARDS_QUERY: &str = include_str!("queries/shards/delete.sql"); + request: PublishSplitsRequest, + ) -> MetastoreResult { + let checkpoint_delta_opt: Option = + request.deserialize_index_checkpoint()?; + let index_uid: IndexUid = request.index_uid().clone(); + let staged_split_ids = request.staged_split_ids; + let replaced_split_ids = request.replaced_split_ids; - const FIND_NOT_DELETABLE_SHARDS_QUERY: &str = - include_str!("queries/shards/find_not_deletable.sql"); + run_with_tx!(self.connection_pool, tx, "publish splits", { + let mut index_metadata = index_metadata(tx, &index_uid.index_id, true).await?; + if index_metadata.index_uid != index_uid { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + if let Some(checkpoint_delta) = checkpoint_delta_opt { + let source_id = checkpoint_delta.source_id.clone(); + let source = index_metadata.sources.get(&source_id).ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Source { + index_id: index_uid.index_id.to_string(), + source_id: source_id.to_string(), + }) + })?; - if request.shard_ids.is_empty() { - return Ok(Default::default()); - } - let query_result = sqlx::query(DELETE_SHARDS_QUERY) - .bind(request.index_uid()) - .bind(&request.source_id) - .bind(&request.shard_ids) - .bind(request.force) - .execute(&self.connection_pool) - .await?; + if use_shard_api(&source.source_params) { + let publish_token = request.publish_token_opt.ok_or_else(|| { + let message = format!( + "publish token is required for publishing splits for source \ + `{source_id}`" + ); + MetastoreError::InvalidArgument { message } + })?; + try_apply_delta_v2( + tx, + &index_uid, + &source_id, + checkpoint_delta.source_delta, + publish_token, + ) + .await?; + } else { + index_metadata + .checkpoint + .try_apply_delta(checkpoint_delta) + .map_err(|error| { + let entity = EntityKind::CheckpointDelta { + index_id: index_uid.index_id.to_string(), + source_id, + }; + let message = error.to_string(); + MetastoreError::FailedPrecondition { entity, message } + })?; + } + } + let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; - // Happy path: all shards were deleted. - if request.force || query_result.rows_affected() == request.shard_ids.len() as u64 { - let response = DeleteShardsResponse { - index_uid: request.index_uid, - source_id: request.source_id, - successes: request.shard_ids, - failures: Vec::new(), - }; - return Ok(response); - } - // Unhappy path: some shards were not deleted because they do not exist or are not fully - // indexed. - let not_deletable_pg_shards: Vec = sqlx::query_as(FIND_NOT_DELETABLE_SHARDS_QUERY) - .bind(request.index_uid()) - .bind(&request.source_id) - .bind(&request.shard_ids) + const PUBLISH_SPLITS_QUERY: &str = r#" + -- Select the splits to update, regardless of their state. + -- The left join make it possible to identify the splits that do not exist. + WITH input_splits AS ( + SELECT input_splits.split_id, input_splits.expected_split_state, splits.actual_split_state + FROM ( + SELECT split_id, 'Staged' AS expected_split_state + FROM UNNEST($3) AS staged_splits(split_id) + UNION + SELECT split_id, 'Published' AS expected_split_state + FROM UNNEST($4) AS published_splits(split_id) + ) input_splits + LEFT JOIN ( + SELECT split_id, split_state AS actual_split_state + FROM splits + WHERE + index_uid = $1 + AND (split_id = ANY($3) OR split_id = ANY($4)) + FOR UPDATE + ) AS splits + USING (split_id) + ), + -- Update the index metadata with the new checkpoint. + updated_index_metadata AS ( + UPDATE indexes + SET + index_metadata_json = $2 + WHERE + index_uid = $1 + AND NOT EXISTS ( + SELECT 1 + FROM input_splits + WHERE + actual_split_state != expected_split_state + ) + ), + -- Publish the staged splits and mark the published splits for deletion. + updated_splits AS ( + UPDATE splits + SET + split_state = CASE split_state + WHEN 'Staged' THEN 'Published' + ELSE 'MarkedForDeletion' + END, + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), + publish_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + FROM input_splits + WHERE + splits.index_uid = $1 + AND splits.split_id = input_splits.split_id + AND NOT EXISTS ( + SELECT 1 + FROM input_splits + WHERE + actual_split_state != expected_split_state + ) + ) + -- Report the outcome of the update query. + SELECT + COUNT(1) FILTER (WHERE actual_split_state = 'Staged' AND expected_split_state = 'Staged'), + COUNT(1) FILTER (WHERE actual_split_state = 'Published' AND expected_split_state = 'Published'), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state IS NULL), ARRAY[]::TEXT[]), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state != 'Staged' AND expected_split_state = 'Staged'), ARRAY[]::TEXT[]), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE actual_split_state != 'Published' AND expected_split_state = 'Published'), ARRAY[]::TEXT[]) + FROM input_splits + "#; + let ( + num_published_splits, + num_marked_splits, + not_found_split_ids, + not_staged_split_ids, + not_marked_split_ids, + ): (i64, i64, Vec, Vec, Vec) = + sqlx::query_as(PUBLISH_SPLITS_QUERY) + .bind(&index_uid) + .bind(index_metadata_json) + .bind(staged_split_ids) + .bind(replaced_split_ids) + .fetch_one(tx.as_mut()) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + + if !not_found_split_ids.is_empty() { + return Err(MetastoreError::NotFound(EntityKind::Splits { + split_ids: not_found_split_ids, + })); + } + if !not_staged_split_ids.is_empty() { + let entity = EntityKind::Splits { + split_ids: not_staged_split_ids, + }; + let message = "splits are not staged".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + if !not_marked_split_ids.is_empty() { + let entity = EntityKind::Splits { + split_ids: not_marked_split_ids, + }; + let message = "splits are not marked for deletion".to_string(); + return Err(MetastoreError::FailedPrecondition { entity, message }); + } + info!( + %index_uid, + "published {num_published_splits} splits and marked {num_marked_splits} for deletion successfully" + ); + Ok(EmptyResponse {}) + }) + } + + #[instrument(skip(self))] + async fn list_splits( + &self, + request: ListSplitsRequest, + ) -> MetastoreResult> { + let list_splits_query = request.deserialize_list_splits_query()?; + let mut sql_query_builder = Query::select(); + sql_query_builder.column(Asterisk).from(Splits::Table); + append_query_filters_and_order_by(&mut sql_query_builder, &list_splits_query); + + let (sql_query, values) = sql_query_builder.build_sqlx(PostgresQueryBuilder); + let pg_split_stream = SplitStream::new( + self.connection_pool.clone(), + sql_query, + |connection_pool: &TrackedPool, sql_query: &String| { + sqlx::query_as_with::<_, PgSplit, _>(sql_query, values).fetch(connection_pool) + }, + ); + let split_stream = + pg_split_stream + .chunks(STREAM_SPLITS_CHUNK_SIZE) + .map(|pg_splits_results| { + let mut splits = Vec::with_capacity(pg_splits_results.len()); + for pg_split_result in pg_splits_results { + let pg_split = match pg_split_result { + Ok(pg_split) => pg_split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to fetch splits".to_string(), + cause: error.to_string(), + }); + } + }; + let split: Split = match pg_split.try_into() { + Ok(split) => split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to convert `PgSplit` to `Split`".to_string(), + cause: error.to_string(), + }); + } + }; + splits.push(split); + } + ListSplitsResponse::try_from_splits(splits) + }); + let service_stream = ServiceStream::new(Box::pin(split_stream)); + Ok(service_stream) + } + + async fn list_index_stats( + &self, + request: ListIndexStatsRequest, + ) -> MetastoreResult { + let index_pattern_sql = build_index_id_patterns_sql_query(&request.index_id_patterns) + .map_err(|error| MetastoreError::Internal { + message: "failed to build `list_index_stats` SQL query".to_string(), + cause: error.to_string(), + })?; + let sql = format!( + "SELECT + i.index_uid, + s.split_state, + COUNT(s.split_state) AS num_splits, + COALESCE(SUM(s.split_size_bytes)::BIGINT, 0) AS total_size_bytes + FROM ({index_pattern_sql}) i + LEFT JOIN splits s ON s.index_uid = i.index_uid + GROUP BY i.index_uid, s.split_state" + ); + + let rows: Vec<(String, Option, i64, i64)> = sqlx::query_as(&sql) .fetch_all(&self.connection_pool) .await?; - if not_deletable_pg_shards.is_empty() { - let response = DeleteShardsResponse { - index_uid: request.index_uid, - source_id: request.source_id, - successes: request.shard_ids, - failures: Vec::new(), + let mut index_stats = HashMap::new(); + for (index_uid_str, split_state, num_splits, total_size_bytes) in rows { + let Ok(index_uid) = IndexUid::from_str(&index_uid_str) else { + return Err(MetastoreError::Internal { + message: "failed to parse index_uid".to_string(), + cause: index_uid_str.to_string(), + }); }; - return Ok(response); + let stats = index_stats + .entry(index_uid_str) + .or_insert_with(|| IndexStats { + index_uid: Some(index_uid), + staged: Some(SplitStats::default()), + published: Some(SplitStats::default()), + marked_for_deletion: Some(SplitStats::default()), + }); + let num_splits = num_splits as u64; + let total_size_bytes = total_size_bytes as u64; + match split_state.as_deref() { + Some("Staged") => { + stats.staged = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + Some("Published") => { + stats.published = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + Some("MarkedForDeletion") => { + stats.marked_for_deletion = Some(SplitStats { + num_splits, + total_size_bytes, + }); + } + None => {} // if an index has no splits, we can keep the defaults + Some(split_state) => { + return Err(MetastoreError::Internal { + message: "invalid split state".to_string(), + cause: split_state.to_string(), + }); + } + } } - let failures: Vec = not_deletable_pg_shards - .into_iter() - .map(|pg_shard| pg_shard.shard_id) - .collect(); - warn!( - index_uid=%request.index_uid(), - source_id=%request.source_id, - "failed to delete shards `{}`: shards are not fully indexed", - failures.iter().join(", ") - ); - let successes: Vec = request - .shard_ids - .into_iter() - .filter(|shard_id| !failures.contains(shard_id)) - .collect(); - let response = DeleteShardsResponse { - index_uid: request.index_uid, - source_id: request.source_id, - successes, - failures, - }; - Ok(response) + + Ok(ListIndexStatsResponse { + index_stats: index_stats.into_values().collect(), + }) } - async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { - const PRUNE_AGE_SHARDS_QUERY: &str = include_str!("queries/shards/prune_age.sql"); - const PRUNE_COUNT_SHARDS_QUERY: &str = include_str!("queries/shards/prune_count.sql"); + #[instrument(skip(self))] + async fn mark_splits_for_deletion( + &self, + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + const MARK_SPLITS_FOR_DELETION_QUERY: &str = r#" + -- Select the splits to update, regardless of their state. + -- The left join make it possible to identify the splits that do not exist. + WITH input_splits AS ( + SELECT input_splits.split_id, splits.split_state + FROM UNNEST($2) AS input_splits(split_id) + LEFT JOIN ( + SELECT split_id, split_state + FROM splits + WHERE + index_uid = $1 + AND split_id = ANY($2) + FOR UPDATE + ) AS splits + USING (split_id) + ), + -- Mark the staged and published splits for deletion. + marked_splits AS ( + UPDATE splits + SET + split_state = 'MarkedForDeletion', + update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + FROM input_splits + WHERE + splits.index_uid = $1 + AND splits.split_id = input_splits.split_id + AND splits.split_state IN ('Staged', 'Published') + ) + -- Report the outcome of the update query. + SELECT + COUNT(split_state), + COUNT(1) FILTER (WHERE split_state IN ('Staged', 'Published')), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) + FROM input_splits + "#; + let (num_found_splits, num_marked_splits, not_found_split_ids): (i64, i64, Vec) = + sqlx::query_as(MARK_SPLITS_FOR_DELETION_QUERY) + .bind(&index_uid) + .bind(split_ids.clone()) + .fetch_one(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; - if let Some(max_age_secs) = request.max_age_secs { - let limit_datetime = - OffsetDateTime::now_utc() - Duration::from_secs(max_age_secs as u64); - sqlx::query(PRUNE_AGE_SHARDS_QUERY) - .bind(request.index_uid()) - .bind(&request.source_id) - .bind(limit_datetime) - .execute(&self.connection_pool) - .await?; + if num_found_splits == 0 + && index_opt(&self.connection_pool, &index_uid.index_id, false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + info!( + %index_uid, + "Marked {} splits for deletion, among which {} were newly marked.", + split_ids.len() - not_found_split_ids.len(), + num_marked_splits + ); + if !not_found_split_ids.is_empty() { + warn!( + %index_uid, + split_ids=?PrettySample::new(¬_found_split_ids, 5), + "{} splits were not found and could not be marked for deletion.", + not_found_split_ids.len() + ); + } + Ok(EmptyResponse {}) + } + + #[instrument(skip(self))] + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + const DELETE_SPLITS_QUERY: &str = r#" + -- Select the splits to delete, regardless of their state. + -- The left join make it possible to identify the splits that do not exist. + WITH input_splits AS ( + SELECT input_splits.split_id, splits.split_state + FROM UNNEST($2) AS input_splits(split_id) + LEFT JOIN ( + SELECT split_id, split_state + FROM splits + WHERE + index_uid = $1 + AND split_id = ANY($2) + FOR UPDATE + ) AS splits + USING (split_id) + ), + -- Delete the splits if and only if all the splits are marked for deletion. + deleted_splits AS ( + DELETE FROM splits + USING input_splits + WHERE + splits.index_uid = $1 + AND splits.split_id = input_splits.split_id + AND NOT EXISTS ( + SELECT 1 + FROM input_splits + WHERE + split_state IN ('Staged', 'Published') + ) + ) + -- Report the outcome of the delete query. + SELECT + COUNT(split_state), + COUNT(1) FILTER (WHERE split_state = 'MarkedForDeletion'), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IN ('Staged', 'Published')), ARRAY[]::TEXT[]), + COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) + FROM input_splits + "#; + let (num_found_splits, num_deleted_splits, not_deletable_split_ids, not_found_split_ids): ( + i64, + i64, + Vec, + Vec, + ) = sqlx::query_as(DELETE_SPLITS_QUERY) + .bind(&index_uid) + .bind(split_ids) + .fetch_one(&self.connection_pool) + .await + .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + + if num_found_splits == 0 + && index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + if !not_deletable_split_ids.is_empty() { + let message = format!( + "splits `{}` are not deletable", + not_deletable_split_ids.join(", ") + ); + let entity = EntityKind::Splits { + split_ids: not_deletable_split_ids, + }; + return Err(MetastoreError::FailedPrecondition { entity, message }); } + info!(%index_uid, "deleted {} splits from index", num_deleted_splits); - if let Some(max_count) = request.max_count { - sqlx::query(PRUNE_COUNT_SHARDS_QUERY) - .bind(request.index_uid()) - .bind(&request.source_id) - .bind(max_count as i64) - .execute(&self.connection_pool) - .await?; + if !not_found_split_ids.is_empty() { + warn!( + %index_uid, + split_ids=?PrettySample::new(¬_found_split_ids, 5), + "{} splits were not found and could not be deleted.", + not_found_split_ids.len() + ); } Ok(EmptyResponse {}) } - // Index Template API - - async fn create_index_template( - &self, - request: CreateIndexTemplateRequest, - ) -> MetastoreResult { - const INSERT_INDEX_TEMPLATE_QUERY: &str = - include_str!("queries/index_templates/insert.sql"); - const UPSERT_INDEX_TEMPLATE_QUERY: &str = - include_str!("queries/index_templates/upsert.sql"); - - let index_template: IndexTemplate = - serde_utils::from_json_str(&request.index_template_json)?; - - index_template - .validate() - .map_err(|error| MetastoreError::InvalidArgument { - message: format!( - "invalid index template `{}`: `{error}`", - index_template.template_id - ), - })?; - - let mut positive_patterns = Vec::new(); - let mut negative_patterns = Vec::new(); + #[instrument(skip(self))] + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "add source", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + index_metadata.add_source(source_config)?; + Ok(MutationOccurred::Yes(())) + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } - for pattern in &index_template.index_id_patterns { - if let Some(negative_pattern) = pattern.strip_prefix('-') { - negative_patterns.push(negative_pattern.replace('*', "%")); - } else { - positive_patterns.push(pattern.replace('*', "%")); - } - } - if request.overwrite { - sqlx::query(UPSERT_INDEX_TEMPLATE_QUERY) - .bind(&index_template.template_id) - .bind(positive_patterns) - .bind(negative_patterns) - .bind(index_template.priority as i32) - .bind(&request.index_template_json) - .execute(&self.connection_pool) - .await?; + #[instrument(skip(self))] + async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "update source", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + let mutation_occurred = index_metadata.update_source(source_config)?; + Ok(MutationOccurred::from(mutation_occurred)) + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } - return Ok(EmptyResponse {}); - } - let pg_query_result = sqlx::query(INSERT_INDEX_TEMPLATE_QUERY) - .bind(&index_template.template_id) - .bind(positive_patterns) - .bind(negative_patterns) - .bind(index_template.priority as i32) - .bind(&request.index_template_json) - .execute(&self.connection_pool) + #[instrument(skip(self))] + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "toggle source", { + mutate_index_metadata(tx, index_uid, |index_metadata| { + if index_metadata.toggle_source(&request.source_id, request.enable)? { + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + } else { + Ok::<_, MetastoreError>(MutationOccurred::No(())) + } + }) .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } - if pg_query_result.rows_affected() == 0 { - return Err(MetastoreError::AlreadyExists(EntityKind::IndexTemplate { - template_id: index_template.template_id, - })); - } + #[instrument(skip(self))] + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let source_id = request.source_id.clone(); + run_with_tx!(self.connection_pool, tx, "delete source", { + mutate_index_metadata(tx, index_uid.clone(), |index_metadata| { + index_metadata.delete_source(&source_id)?; + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + }) + .await?; + sqlx::query( + r#" + DELETE FROM shards + WHERE + index_uid = $1 + AND source_id = $2 + "#, + ) + .bind(&index_uid) + .bind(source_id) + .execute(tx.as_mut()) + .await?; + Ok(()) + })?; Ok(EmptyResponse {}) } - async fn get_index_template( + #[instrument(skip(self))] + async fn reset_source_checkpoint( &self, - request: GetIndexTemplateRequest, - ) -> MetastoreResult { - let pg_index_template_json: PgIndexTemplate = - sqlx::query_as("SELECT * FROM index_templates WHERE template_id = $1") - .bind(&request.template_id) - .fetch_optional(&self.connection_pool) - .await? - .ok_or({ - MetastoreError::NotFound(EntityKind::IndexTemplate { - template_id: request.template_id, - }) - })?; - let response = GetIndexTemplateResponse { - index_template_json: pg_index_template_json.index_template_json, - }; - Ok(response) + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "reset source checkpoint", { + mutate_index_metadata(tx, index_uid, |index_metadata| { + if index_metadata.checkpoint.reset_source(&request.source_id) { + Ok::<_, MetastoreError>(MutationOccurred::Yes(())) + } else { + Ok::<_, MetastoreError>(MutationOccurred::No(())) + } + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) } - async fn find_index_template_matches( + /// Retrieves the last delete opstamp for a given `index_id`. + #[instrument(skip(self))] + async fn last_delete_opstamp( &self, - request: FindIndexTemplateMatchesRequest, - ) -> MetastoreResult { - if request.index_ids.is_empty() { - return Ok(Default::default()); - } - const FIND_INDEX_TEMPLATE_MATCHES_QUERY: &str = - include_str!("queries/index_templates/find.sql"); + request: LastDeleteOpstampRequest, + ) -> MetastoreResult { + let max_opstamp: i64 = sqlx::query_scalar( + r#" + SELECT COALESCE(MAX(opstamp), 0) + FROM delete_tasks + WHERE index_uid = $1 + "#, + ) + .bind(request.index_uid()) + .fetch_one(&self.connection_pool) + .await + .map_err(|error| MetastoreError::Db { + message: error.to_string(), + })?; - let sql_matches: Vec<(IndexId, IndexTemplateId, String)> = - sqlx::query_as(FIND_INDEX_TEMPLATE_MATCHES_QUERY) - .bind(&request.index_ids) - .fetch_all(&self.connection_pool) - .await?; + Ok(LastDeleteOpstampResponse::new(max_opstamp as u64)) + } - let matches = sql_matches - .into_iter() - .map( - |(index_id, template_id, index_template_json)| IndexTemplateMatch { - index_id, - template_id, - index_template_json, - }, + /// Creates a delete task from a delete query. + #[instrument(skip(self))] + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { + let delete_query_json = serde_utils::to_json_str(&delete_query)?; + let (create_timestamp, opstamp): (sqlx::types::time::PrimitiveDateTime, i64) = + sqlx::query_as( + r#" + INSERT INTO delete_tasks (index_uid, delete_query_json) VALUES ($1, $2) + RETURNING create_timestamp, opstamp + "#, ) - .collect(); - let response = FindIndexTemplateMatchesResponse { matches }; - Ok(response) + .bind(delete_query.index_uid().to_string()) + .bind(&delete_query_json) + .fetch_one(&self.connection_pool) + .await + .map_err(|error| convert_sqlx_err(&delete_query.index_uid().index_id, error))?; + + Ok(DeleteTask { + create_timestamp: create_timestamp.assume_utc().unix_timestamp(), + opstamp: opstamp as u64, + delete_query: Some(delete_query), + }) } - async fn list_index_templates( + /// Update splits delete opstamps. + #[instrument(skip(self))] + async fn update_splits_delete_opstamp( &self, - _request: ListIndexTemplatesRequest, - ) -> MetastoreResult { - let pg_index_templates_json: Vec<(String,)> = sqlx::query_as( - "SELECT index_template_json FROM index_templates ORDER BY template_id ASC", + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let split_ids = request.split_ids; + if split_ids.is_empty() { + return Ok(UpdateSplitsDeleteOpstampResponse {}); + } + let update_result = sqlx::query( + r#" + UPDATE splits + SET + delete_opstamp = $1, + -- The values we compare with are *before* the modification: + update_timestamp = CASE + WHEN delete_opstamp != $1 THEN (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') + ELSE update_timestamp + END + WHERE + index_uid = $2 + AND split_id = ANY($3) + "#, ) - .fetch_all(&self.connection_pool) + .bind(request.delete_opstamp as i64) + .bind(&index_uid) + .bind(split_ids) + .execute(&self.connection_pool) .await?; - let index_templates_json: Vec = pg_index_templates_json - .into_iter() - .map(|(index_template_json,)| index_template_json) - .collect(); - let response = ListIndexTemplatesResponse { - index_templates_json, - }; - Ok(response) - } - async fn delete_index_templates( - &self, - request: DeleteIndexTemplatesRequest, - ) -> MetastoreResult { - sqlx::query("DELETE FROM index_templates WHERE template_id = ANY($1)") - .bind(&request.template_ids) - .execute(&self.connection_pool) - .await?; - Ok(EmptyResponse {}) + // If no splits were updated, maybe the index does not exist in the first place? + if update_result.rows_affected() == 0 + && index_opt_for_uid(&self.connection_pool, index_uid.clone(), false) + .await? + .is_none() + { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.index_id, + })); + } + Ok(UpdateSplitsDeleteOpstampResponse {}) } - async fn get_cluster_identity( + /// Lists the delete tasks with opstamp > `opstamp_start`. + #[instrument(skip(self))] + async fn list_delete_tasks( &self, - _: GetClusterIdentityRequest, - ) -> MetastoreResult { - // `ON CONFLICT DO NOTHING RETURNING` returns NULL if no insert happens. - // To always get the value, we use this pattern: - let (uuid,) = sqlx::query_as( - r" - INSERT INTO kv (key, value) - VALUES ('cluster_identity', $1) - ON CONFLICT (key) DO UPDATE SET key = EXCLUDED.key - RETURNING value - ", + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let pg_delete_tasks: Vec = sqlx::query_as::<_, PgDeleteTask>( + r#" + SELECT * FROM delete_tasks + WHERE + index_uid = $1 + AND opstamp > $2 + "#, ) - .bind(Uuid::new_v4().hyphenated().to_string()) - .fetch_one(&self.connection_pool) + .bind(&index_uid) + .bind(request.opstamp_start as i64) + .fetch_all(&self.connection_pool) .await?; - Ok(GetClusterIdentityResponse { uuid }) + let delete_tasks: Vec = pg_delete_tasks + .into_iter() + .map(|pg_delete_task| pg_delete_task.try_into()) + .collect::>()?; + Ok(ListDeleteTasksResponse { delete_tasks }) } - // Metrics Splits API - - async fn stage_metrics_splits( - &self, - request: StageMetricsSplitsRequest, - ) -> MetastoreResult { - use quickwit_parquet_engine::split::{ - InsertableMetricsSplit, MetricsSplitMetadata, MetricsSplitState, - }; - - let index_uid = request.index_uid().clone(); - - // Deserialize the splits metadata - let splits_metadata: Vec = request.deserialize_splits_metadata()?; - - if splits_metadata.is_empty() { - return Ok(EmptyResponse {}); - } - - // Convert to insertable rows - use JSON strings for array fields due to sqlx limitations - let mut split_ids = Vec::with_capacity(splits_metadata.len()); - let mut split_states = Vec::with_capacity(splits_metadata.len()); - let mut index_uids = Vec::with_capacity(splits_metadata.len()); - let mut time_range_starts = Vec::with_capacity(splits_metadata.len()); - let mut time_range_ends = Vec::with_capacity(splits_metadata.len()); - let mut metric_names_json = Vec::with_capacity(splits_metadata.len()); - let mut tag_service_json = Vec::with_capacity(splits_metadata.len()); - let mut tag_env_json = Vec::with_capacity(splits_metadata.len()); - let mut tag_datacenter_json = Vec::with_capacity(splits_metadata.len()); - let mut tag_region_json = Vec::with_capacity(splits_metadata.len()); - let mut tag_host_json = Vec::with_capacity(splits_metadata.len()); - let mut high_card_tag_keys_json = Vec::with_capacity(splits_metadata.len()); - let mut num_rows_list = Vec::with_capacity(splits_metadata.len()); - let mut size_bytes_list = Vec::with_capacity(splits_metadata.len()); - let mut split_metadata_jsons = Vec::with_capacity(splits_metadata.len()); - - for metadata in &splits_metadata { - let insertable = - InsertableMetricsSplit::from_metadata(metadata, MetricsSplitState::Staged) - .map_err(|e| MetastoreError::JsonSerializeError { - struct_name: "MetricsSplitMetadata".to_string(), - message: e.to_string(), - })?; - - split_ids.push(insertable.split_id); - split_states.push(insertable.split_state); - index_uids.push(insertable.index_uid); - time_range_starts.push(insertable.time_range_start); - time_range_ends.push(insertable.time_range_end); - // Convert array fields to JSON strings (owned) - let json_err = |e: serde_json::Error| MetastoreError::JsonSerializeError { - struct_name: "MetricsSplitMetadata".to_string(), - message: e.to_string(), - }; - metric_names_json - .push(serde_json::to_string(&insertable.metric_names).map_err(json_err)?); - tag_service_json - .push(serde_json::to_string(&insertable.tag_service).map_err(json_err)?); - tag_env_json.push(serde_json::to_string(&insertable.tag_env).map_err(json_err)?); - tag_datacenter_json - .push(serde_json::to_string(&insertable.tag_datacenter).map_err(json_err)?); - tag_region_json.push(serde_json::to_string(&insertable.tag_region).map_err(json_err)?); - tag_host_json.push(serde_json::to_string(&insertable.tag_host).map_err(json_err)?); - high_card_tag_keys_json.push( - serde_json::to_string(&insertable.high_cardinality_tag_keys).map_err(json_err)?, - ); - num_rows_list.push(insertable.num_rows); - size_bytes_list.push(insertable.size_bytes); - split_metadata_jsons.push(insertable.split_metadata_json); - } - - info!( - index_uid = %request.index_uid(), - num_splits = split_ids.len(), - "staging metrics splits" - ); - - // Insert into metrics_splits table using JSON for array fields - const STAGE_METRICS_SPLITS_QUERY: &str = r#" - INSERT INTO metrics_splits ( - split_id, - split_state, - index_uid, - time_range_start, - time_range_end, - metric_names, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - high_cardinality_tag_keys, - num_rows, - size_bytes, - split_metadata_json, - create_timestamp, - update_timestamp - ) - SELECT - split_id, - split_state, - index_uid, - time_range_start, - time_range_end, - ARRAY(SELECT json_array_elements_text(metric_names_json::json)), - CASE WHEN tag_service_json::text = 'null' THEN NULL - ELSE ARRAY(SELECT json_array_elements_text(tag_service_json::json)) END, - CASE WHEN tag_env_json::text = 'null' THEN NULL - ELSE ARRAY(SELECT json_array_elements_text(tag_env_json::json)) END, - CASE WHEN tag_datacenter_json::text = 'null' THEN NULL - ELSE ARRAY(SELECT json_array_elements_text(tag_datacenter_json::json)) END, - CASE WHEN tag_region_json::text = 'null' THEN NULL - ELSE ARRAY(SELECT json_array_elements_text(tag_region_json::json)) END, - CASE WHEN tag_host_json::text = 'null' THEN NULL - ELSE ARRAY(SELECT json_array_elements_text(tag_host_json::json)) END, - ARRAY(SELECT json_array_elements_text(high_cardinality_tag_keys_json::json)), - num_rows, - size_bytes, - split_metadata_json, - (CURRENT_TIMESTAMP AT TIME ZONE 'UTC'), - (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - FROM UNNEST( - $1::text[], - $2::text[], - $3::text[], - $4::bigint[], - $5::bigint[], - $6::text[], - $7::text[], - $8::text[], - $9::text[], - $10::text[], - $11::text[], - $12::text[], - $13::bigint[], - $14::bigint[], - $15::text[] - ) AS staged( - split_id, - split_state, - index_uid, - time_range_start, - time_range_end, - metric_names_json, - tag_service_json, - tag_env_json, - tag_datacenter_json, - tag_region_json, - tag_host_json, - high_cardinality_tag_keys_json, - num_rows, - size_bytes, - split_metadata_json - ) - ON CONFLICT (split_id) DO UPDATE - SET - split_state = EXCLUDED.split_state, - time_range_start = EXCLUDED.time_range_start, - time_range_end = EXCLUDED.time_range_end, - metric_names = EXCLUDED.metric_names, - tag_service = EXCLUDED.tag_service, - tag_env = EXCLUDED.tag_env, - tag_datacenter = EXCLUDED.tag_datacenter, - tag_region = EXCLUDED.tag_region, - tag_host = EXCLUDED.tag_host, - high_cardinality_tag_keys = EXCLUDED.high_cardinality_tag_keys, - num_rows = EXCLUDED.num_rows, - size_bytes = EXCLUDED.size_bytes, - split_metadata_json = EXCLUDED.split_metadata_json, - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - WHERE metrics_splits.split_state = 'Staged' - RETURNING split_id - "#; + /// Returns `num_splits` published splits with `split.delete_opstamp` < `delete_opstamp`. + /// Results are ordered by ascending `split.delete_opstamp` and `split.publish_timestamp` + /// values. + #[instrument(skip(self))] + async fn list_stale_splits( + &self, + request: ListStaleSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid().clone(); + let stale_pg_splits: Vec = sqlx::query_as::<_, PgSplit>( + r#" + SELECT * + FROM splits + WHERE + index_uid = $1 + AND delete_opstamp < $2 + AND split_state = $3 + AND (maturity_timestamp = to_timestamp(0) OR (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') >= maturity_timestamp) + ORDER BY delete_opstamp ASC, publish_timestamp ASC + LIMIT $4 + "#, + ) + .bind(&index_uid) + .bind(request.delete_opstamp as i64) + .bind(SplitState::Published.as_str()) + .bind(request.num_splits as i64) + .fetch_all(&self.connection_pool) + .await?; - let split_ids_snapshot = split_ids.clone(); - let index_id_for_err = index_uid.index_id.clone(); - let upserted_split_ids: Vec = - run_with_tx!(self.connection_pool, tx, "stage metrics splits", { - sqlx::query_scalar(STAGE_METRICS_SPLITS_QUERY) - .bind(&split_ids) - .bind(&split_states) - .bind(&index_uids) - .bind(&time_range_starts) - .bind(&time_range_ends) - .bind(&metric_names_json) - .bind(&tag_service_json) - .bind(&tag_env_json) - .bind(&tag_datacenter_json) - .bind(&tag_region_json) - .bind(&tag_host_json) - .bind(&high_card_tag_keys_json) - .bind(&num_rows_list) - .bind(&size_bytes_list) - .bind(&split_metadata_jsons) - .fetch_all(tx.as_mut()) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_id_for_err, sqlx_error)) - })?; + let stale_splits: Vec = stale_pg_splits + .into_iter() + .map(|pg_split| pg_split.try_into()) + .collect::>()?; + let response = ListSplitsResponse::try_from_splits(stale_splits)?; + Ok(response) + } - if upserted_split_ids.len() != split_ids_snapshot.len() { - let failed_split_ids: Vec = split_ids_snapshot - .into_iter() - .filter(|split_id| !upserted_split_ids.contains(split_id)) - .collect(); - let entity = EntityKind::Splits { - split_ids: failed_split_ids.clone(), + // TODO: Issue a single SQL query. + async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { + let mut subresponses = Vec::with_capacity(request.subrequests.len()); + + for subrequest in request.subrequests { + let open_shard: Shard = open_or_fetch_shard(&self.connection_pool, &subrequest).await?; + let subresponse = OpenShardSubresponse { + subrequest_id: subrequest.subrequest_id, + open_shard: Some(open_shard), }; - let message = "splits are not in the Staged state".to_string(); - warn!( - %index_uid, - failed_split_ids = ?failed_split_ids, - "failed to stage some metrics splits" - ); - return Err(MetastoreError::FailedPrecondition { entity, message }); + subresponses.push(subresponse); } - - info!( - %index_uid, - num_staged = upserted_split_ids.len(), - "staged metrics splits successfully" - ); - Ok(EmptyResponse {}) + Ok(OpenShardsResponse { subresponses }) } - #[instrument(skip_all, fields(index_uid = %request.index_uid()))] - async fn publish_metrics_splits( + async fn acquire_shards( &self, - request: PublishMetricsSplitsRequest, - ) -> MetastoreResult { - let checkpoint_delta_opt: Option = - request.deserialize_index_checkpoint()?; - let index_uid = request.index_uid().clone(); - let staged_split_ids = request.staged_split_ids; - let replaced_split_ids = request.replaced_split_ids; - - info!( - %index_uid, - staged_splits = staged_split_ids.len(), - replaced_splits = replaced_split_ids.len(), - "publishing metrics splits" - ); + request: AcquireShardsRequest, + ) -> MetastoreResult { + const ACQUIRE_SHARDS_QUERY: &str = include_str!("queries/shards/acquire.sql"); - run_with_tx!(self.connection_pool, tx, "publish metrics splits", { - let mut index_metadata = index_metadata(tx, &index_uid.index_id, true).await?; - let index_uid = index_metadata.index_uid.clone(); + if request.shard_ids.is_empty() { + return Ok(Default::default()); + } + let pg_shards: Vec = sqlx::query_as(ACQUIRE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(&request.shard_ids) + .bind(&request.publish_token) + .fetch_all(&self.connection_pool) + .await?; + let acquired_shards = pg_shards + .into_iter() + .map(|pg_shard| pg_shard.into()) + .collect(); + let response = AcquireShardsResponse { acquired_shards }; + Ok(response) + } - if let Some(checkpoint_delta) = checkpoint_delta_opt { - let source_id = checkpoint_delta.source_id.clone(); - let source = index_metadata.sources.get(&source_id).ok_or_else(|| { - MetastoreError::NotFound(EntityKind::Source { - index_id: index_uid.index_id.to_string(), - source_id: source_id.to_string(), - }) - })?; + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { + if request.subrequests.is_empty() { + return Ok(Default::default()); + } + let mut sql_query_builder = Query::select(); - if use_shard_api(&source.source_params) { - let publish_token = request.publish_token_opt.ok_or_else(|| { - let message = format!( - "publish token is required for publishing splits for source \ - `{source_id}`" - ); - MetastoreError::InvalidArgument { message } - })?; - try_apply_delta_v2( - tx, - &index_uid, - &source_id, - checkpoint_delta.source_delta, - publish_token, - ) - .await?; - } else { - index_metadata - .checkpoint - .try_apply_delta(checkpoint_delta) - .map_err(|error| { - let entity = EntityKind::CheckpointDelta { - index_id: index_uid.index_id.to_string(), - source_id, - }; - let message = error.to_string(); - MetastoreError::FailedPrecondition { entity, message } - })?; - } - } - let index_metadata_json = serde_utils::to_json_str(&index_metadata)?; + for (idx, subrequest) in request.subrequests.iter().enumerate() { + let mut sql_subquery_builder = Query::select(); - // Publish staged splits, mark replaced splits for deletion, and update - // index metadata (checkpoint) atomically. - const PUBLISH_METRICS_SPLITS_QUERY: &str = r#" - WITH publish AS ( - UPDATE metrics_splits - SET - split_state = 'Published', - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - WHERE - index_uid = $1 - AND split_id = ANY($3) - AND split_state = 'Staged' - RETURNING split_id - ), - mark_for_deletion AS ( - UPDATE metrics_splits - SET - split_state = 'MarkedForDeletion', - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - WHERE - index_uid = $1 - AND split_id = ANY($4) - AND split_state = 'Published' - RETURNING split_id - ), - updated_index_metadata AS ( - UPDATE indexes - SET - index_metadata_json = $2 - WHERE - index_uid = $1 - ) - SELECT - (SELECT COUNT(*) FROM publish) as published_count, - (SELECT COUNT(*) FROM mark_for_deletion) as marked_count - "#; + sql_subquery_builder + .column(Asterisk) + .from(Shards::Table) + .and_where(Expr::col(Shards::IndexUid).eq(subrequest.index_uid())) + .and_where(Expr::col(Shards::SourceId).eq(&subrequest.source_id)); - let (published_count, marked_count): (i64, i64) = - sqlx::query_as(PUBLISH_METRICS_SPLITS_QUERY) - .bind(&index_uid) - .bind(index_metadata_json) - .bind(&staged_split_ids) - .bind(&replaced_split_ids) - .fetch_one(tx.as_mut()) - .await - .map_err(|sqlx_error| convert_sqlx_err(&index_uid.index_id, sqlx_error))?; + let shard_state = subrequest.shard_state(); - // Verify all staged splits were published - if published_count as usize != staged_split_ids.len() { - let entity = EntityKind::Splits { - split_ids: staged_split_ids.clone(), - }; - let message = format!( - "expected to publish {} splits, but only {} were in Staged state", - staged_split_ids.len(), - published_count - ); - return Err(MetastoreError::FailedPrecondition { entity, message }); + if shard_state != ShardState::Unspecified { + let shard_state_str = shard_state.as_json_str_name(); + let shard_state_alias = Alias::new("SHARD_STATE"); + let cast_expr = Func::cast_as(shard_state_str, shard_state_alias); + sql_subquery_builder.and_where(Expr::col(Shards::ShardState).eq(cast_expr)); } - - // Verify all replaced splits were marked for deletion - if marked_count as usize != replaced_split_ids.len() { - let entity = EntityKind::Splits { - split_ids: replaced_split_ids.clone(), - }; - let message = format!( - "expected to mark {} splits for deletion, but only {} were in Published state", - replaced_split_ids.len(), - marked_count - ); - return Err(MetastoreError::FailedPrecondition { entity, message }); + if idx == 0 { + sql_query_builder = sql_subquery_builder; + } else { + sql_query_builder.union(UnionType::All, sql_subquery_builder); } + } + let (sql_query, values) = sql_query_builder.build_sqlx(PostgresQueryBuilder); + + let pg_shards: Vec = sqlx::query_as_with::<_, PgShard, _>(&sql_query, values) + .fetch_all(&self.connection_pool) + .await?; + + let mut per_source_subresponses: HashMap<(IndexUid, SourceId), ListShardsSubresponse> = + request + .subrequests + .into_iter() + .map(|subrequest| { + let index_uid = subrequest.index_uid().clone(); + let source_id = subrequest.source_id.clone(); + ( + (index_uid, source_id), + ListShardsSubresponse { + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shards: Vec::new(), + }, + ) + }) + .collect(); + + for pg_shard in pg_shards { + let shard: Shard = pg_shard.into(); + let source_key = (shard.index_uid().clone(), shard.source_id.clone()); - info!( - %index_uid, - published_count, - marked_count, - "published metrics splits successfully" - ); - Ok(EmptyResponse {}) - }) + let Some(subresponse) = per_source_subresponses.get_mut(&source_key) else { + warn!( + index_uid=%shard.index_uid(), + source_id=%shard.source_id, + "could not find source in subresponses: this should never happen, please report" + ); + continue; + }; + subresponse.shards.push(shard); + } + let subresponses = per_source_subresponses.into_values().collect(); + let response = ListShardsResponse { subresponses }; + Ok(response) } - #[instrument(skip_all, fields(index_uid = %request.index_uid()))] - async fn list_metrics_splits( + async fn delete_shards( &self, - request: ListMetricsSplitsRequest, - ) -> MetastoreResult { - use quickwit_parquet_engine::split::MetricsSplitRecord; - - use crate::metastore::ListMetricsSplitsQuery; - - // Deserialize the query - let query: ListMetricsSplitsQuery = serde_utils::from_json_str(&request.query_json)?; - - // Build dynamic SQL query based on filters - let mut sql = String::from( - r#" - SELECT - split_id, - split_state, - index_uid, - time_range_start, - time_range_end, - metric_names, - tag_service, - tag_env, - tag_datacenter, - tag_region, - tag_host, - high_cardinality_tag_keys, - num_rows, - size_bytes, - split_metadata_json, - EXTRACT(EPOCH FROM update_timestamp)::bigint as update_timestamp - FROM metrics_splits - WHERE index_uid = $1 - "#, - ); + request: DeleteShardsRequest, + ) -> MetastoreResult { + const DELETE_SHARDS_QUERY: &str = include_str!("queries/shards/delete.sql"); - let mut param_idx = 2; + const FIND_NOT_DELETABLE_SHARDS_QUERY: &str = + include_str!("queries/shards/find_not_deletable.sql"); - // Add state filter - if !query.split_states.is_empty() { - sql.push_str(&format!(" AND split_state = ANY(${}::text[])", param_idx)); - param_idx += 1; + if request.shard_ids.is_empty() { + return Ok(Default::default()); } + let query_result = sqlx::query(DELETE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(&request.shard_ids) + .bind(request.force) + .execute(&self.connection_pool) + .await?; - // Add time range filter - if query.time_range_start.is_some() { - sql.push_str(&format!(" AND time_range_end >= ${}", param_idx)); - param_idx += 1; - } - if query.time_range_end.is_some() { - sql.push_str(&format!(" AND time_range_start <= ${}", param_idx)); - param_idx += 1; + // Happy path: all shards were deleted. + if request.force || query_result.rows_affected() == request.shard_ids.len() as u64 { + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes: request.shard_ids, + failures: Vec::new(), + }; + return Ok(response); } + // Unhappy path: some shards were not deleted because they do not exist or are not fully + // indexed. + let not_deletable_pg_shards: Vec = sqlx::query_as(FIND_NOT_DELETABLE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(&request.shard_ids) + .fetch_all(&self.connection_pool) + .await?; - // Add metric names filter (ANY overlap) - if !query.metric_names.is_empty() { - sql.push_str(&format!(" AND metric_names && ${}::text[]", param_idx)); - param_idx += 1; + if not_deletable_pg_shards.is_empty() { + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes: request.shard_ids, + failures: Vec::new(), + }; + return Ok(response); } + let failures: Vec = not_deletable_pg_shards + .into_iter() + .map(|pg_shard| pg_shard.shard_id) + .collect(); + warn!( + index_uid=%request.index_uid(), + source_id=%request.source_id, + "failed to delete shards `{}`: shards are not fully indexed", + failures.iter().join(", ") + ); + let successes: Vec = request + .shard_ids + .into_iter() + .filter(|shard_id| !failures.contains(shard_id)) + .collect(); + let response = DeleteShardsResponse { + index_uid: request.index_uid, + source_id: request.source_id, + successes, + failures, + }; + Ok(response) + } - // Add tag filters - if query.tag_service.is_some() { - sql.push_str(&format!(" AND ${} = ANY(tag_service)", param_idx)); - param_idx += 1; - } - if query.tag_env.is_some() { - sql.push_str(&format!(" AND ${} = ANY(tag_env)", param_idx)); - param_idx += 1; - } - if query.tag_datacenter.is_some() { - sql.push_str(&format!(" AND ${} = ANY(tag_datacenter)", param_idx)); - param_idx += 1; - } - if query.tag_region.is_some() { - sql.push_str(&format!(" AND ${} = ANY(tag_region)", param_idx)); - param_idx += 1; + async fn prune_shards(&self, request: PruneShardsRequest) -> MetastoreResult { + const PRUNE_AGE_SHARDS_QUERY: &str = include_str!("queries/shards/prune_age.sql"); + const PRUNE_COUNT_SHARDS_QUERY: &str = include_str!("queries/shards/prune_count.sql"); + + if let Some(max_age_secs) = request.max_age_secs { + let limit_datetime = + OffsetDateTime::now_utc() - Duration::from_secs(max_age_secs as u64); + sqlx::query(PRUNE_AGE_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(limit_datetime) + .execute(&self.connection_pool) + .await?; } - if query.tag_host.is_some() { - sql.push_str(&format!(" AND ${} = ANY(tag_host)", param_idx)); - param_idx += 1; + + if let Some(max_count) = request.max_count { + sqlx::query(PRUNE_COUNT_SHARDS_QUERY) + .bind(request.index_uid()) + .bind(&request.source_id) + .bind(max_count as i64) + .execute(&self.connection_pool) + .await?; } + Ok(EmptyResponse {}) + } - sql.push_str(" ORDER BY time_range_start ASC"); + // Index Template API - // Add limit - if query.limit.is_some() { - sql.push_str(&format!(" LIMIT ${}", param_idx)); - } + async fn create_index_template( + &self, + request: CreateIndexTemplateRequest, + ) -> MetastoreResult { + const INSERT_INDEX_TEMPLATE_QUERY: &str = + include_str!("queries/index_templates/insert.sql"); + const UPSERT_INDEX_TEMPLATE_QUERY: &str = + include_str!("queries/index_templates/upsert.sql"); - // Execute query with bindings - let mut query_builder = sqlx::query_as::< - _, - ( - String, // split_id - String, // split_state - String, // index_uid - i64, // time_range_start - i64, // time_range_end - Vec, // metric_names - Option>, // tag_service - Option>, // tag_env - Option>, // tag_datacenter - Option>, // tag_region - Option>, // tag_host - Vec, // high_cardinality_tag_keys - i64, // num_rows - i64, // size_bytes - String, // split_metadata_json - i64, // update_timestamp - ), - >(&sql); + let index_template: IndexTemplate = + serde_utils::from_json_str(&request.index_template_json)?; - query_builder = query_builder.bind(query.index_uid.to_string()); + index_template + .validate() + .map_err(|error| MetastoreError::InvalidArgument { + message: format!( + "invalid index template `{}`: `{error}`", + index_template.template_id + ), + })?; - if !query.split_states.is_empty() { - query_builder = query_builder.bind(&query.split_states); - } - if let Some(start) = query.time_range_start { - query_builder = query_builder.bind(start); - } - if let Some(end) = query.time_range_end { - query_builder = query_builder.bind(end); - } - if !query.metric_names.is_empty() { - query_builder = query_builder.bind(&query.metric_names); - } - if let Some(ref service) = query.tag_service { - query_builder = query_builder.bind(service); - } - if let Some(ref env) = query.tag_env { - query_builder = query_builder.bind(env); - } - if let Some(ref dc) = query.tag_datacenter { - query_builder = query_builder.bind(dc); - } - if let Some(ref region) = query.tag_region { - query_builder = query_builder.bind(region); - } - if let Some(ref host) = query.tag_host { - query_builder = query_builder.bind(host); + let mut positive_patterns = Vec::new(); + let mut negative_patterns = Vec::new(); + + for pattern in &index_template.index_id_patterns { + if let Some(negative_pattern) = pattern.strip_prefix('-') { + negative_patterns.push(negative_pattern.replace('*', "%")); + } else { + positive_patterns.push(pattern.replace('*', "%")); + } } - if let Some(limit) = query.limit { - query_builder = query_builder.bind(limit as i64); + if request.overwrite { + sqlx::query(UPSERT_INDEX_TEMPLATE_QUERY) + .bind(&index_template.template_id) + .bind(positive_patterns) + .bind(negative_patterns) + .bind(index_template.priority as i32) + .bind(&request.index_template_json) + .execute(&self.connection_pool) + .await?; + + return Ok(EmptyResponse {}); } + let pg_query_result = sqlx::query(INSERT_INDEX_TEMPLATE_QUERY) + .bind(&index_template.template_id) + .bind(positive_patterns) + .bind(negative_patterns) + .bind(index_template.priority as i32) + .bind(&request.index_template_json) + .execute(&self.connection_pool) + .await?; - let rows = query_builder - .fetch_all(&self.connection_pool) - .await - .map_err(|sqlx_error| convert_sqlx_err(&query.index_uid.index_id, sqlx_error))?; + if pg_query_result.rows_affected() == 0 { + return Err(MetastoreError::AlreadyExists(EntityKind::IndexTemplate { + template_id: index_template.template_id, + })); + } + Ok(EmptyResponse {}) + } - // Convert rows to MetricsSplitRecord - let splits: Vec = rows - .into_iter() - .filter_map(|row| { - use quickwit_parquet_engine::split::{MetricsSplitState, PgMetricsSplit}; + async fn get_index_template( + &self, + request: GetIndexTemplateRequest, + ) -> MetastoreResult { + let pg_index_template_json: PgIndexTemplate = + sqlx::query_as("SELECT * FROM index_templates WHERE template_id = $1") + .bind(&request.template_id) + .fetch_optional(&self.connection_pool) + .await? + .ok_or({ + MetastoreError::NotFound(EntityKind::IndexTemplate { + template_id: request.template_id, + }) + })?; + let response = GetIndexTemplateResponse { + index_template_json: pg_index_template_json.index_template_json, + }; + Ok(response) + } - let pg_split = PgMetricsSplit { - split_id: row.0, - split_state: row.1, - index_uid: row.2, - time_range_start: row.3, - time_range_end: row.4, - metric_names: row.5, - tag_service: row.6, - tag_env: row.7, - tag_datacenter: row.8, - tag_region: row.9, - tag_host: row.10, - high_cardinality_tag_keys: row.11, - num_rows: row.12, - size_bytes: row.13, - split_metadata_json: row.14, - update_timestamp: row.15, - }; + async fn find_index_template_matches( + &self, + request: FindIndexTemplateMatchesRequest, + ) -> MetastoreResult { + if request.index_ids.is_empty() { + return Ok(Default::default()); + } + const FIND_INDEX_TEMPLATE_MATCHES_QUERY: &str = + include_str!("queries/index_templates/find.sql"); - let state = pg_split.split_state().unwrap_or(MetricsSplitState::Staged); - let metadata = pg_split.to_metadata().ok()?; + let sql_matches: Vec<(IndexId, IndexTemplateId, String)> = + sqlx::query_as(FIND_INDEX_TEMPLATE_MATCHES_QUERY) + .bind(&request.index_ids) + .fetch_all(&self.connection_pool) + .await?; - Some(MetricsSplitRecord { - state, - update_timestamp: row.15, - metadata, - }) - }) + let matches = sql_matches + .into_iter() + .map( + |(index_id, template_id, index_template_json)| IndexTemplateMatch { + index_id, + template_id, + index_template_json, + }, + ) .collect(); + let response = FindIndexTemplateMatchesResponse { matches }; + Ok(response) + } - ListMetricsSplitsResponse::try_from_splits(&splits) + async fn list_index_templates( + &self, + _request: ListIndexTemplatesRequest, + ) -> MetastoreResult { + let pg_index_templates_json: Vec<(String,)> = sqlx::query_as( + "SELECT index_template_json FROM index_templates ORDER BY template_id ASC", + ) + .fetch_all(&self.connection_pool) + .await?; + let index_templates_json: Vec = pg_index_templates_json + .into_iter() + .map(|(index_template_json,)| index_template_json) + .collect(); + let response = ListIndexTemplatesResponse { + index_templates_json, + }; + Ok(response) } - #[instrument(skip_all, fields(index_uid = %request.index_uid()))] - async fn mark_metrics_splits_for_deletion( + async fn delete_index_templates( &self, - request: MarkMetricsSplitsForDeletionRequest, + request: DeleteIndexTemplatesRequest, ) -> MetastoreResult { - if request.split_ids.is_empty() { - return Ok(EmptyResponse {}); - } + sqlx::query("DELETE FROM index_templates WHERE template_id = ANY($1)") + .bind(&request.template_ids) + .execute(&self.connection_pool) + .await?; + Ok(EmptyResponse {}) + } - info!( - index_uid = %request.index_uid(), - split_ids = ?request.split_ids, - "marking metrics splits for deletion" - ); + async fn get_cluster_identity( + &self, + _: GetClusterIdentityRequest, + ) -> MetastoreResult { + // `ON CONFLICT DO NOTHING RETURNING` returns NULL if no insert happens. + // To always get the value, we use this pattern: + let (uuid,) = sqlx::query_as( + r" + INSERT INTO kv (key, value) + VALUES ('cluster_identity', $1) + ON CONFLICT (key) DO UPDATE SET key = EXCLUDED.key + RETURNING value + ", + ) + .bind(Uuid::new_v4().hyphenated().to_string()) + .fetch_one(&self.connection_pool) + .await?; + Ok(GetClusterIdentityResponse { uuid }) + } - const MARK_FOR_DELETION_QUERY: &str = r#" - UPDATE metrics_splits - SET - split_state = 'MarkedForDeletion', - update_timestamp = (CURRENT_TIMESTAMP AT TIME ZONE 'UTC') - WHERE - index_uid = $1 - AND split_id = ANY($2) - AND split_state IN ('Staged', 'Published') - RETURNING split_id - "#; + // Metrics Splits API - let marked_split_ids: Vec = sqlx::query_scalar(MARK_FOR_DELETION_QUERY) - .bind(request.index_uid()) - .bind(&request.split_ids) - .fetch_all(&self.connection_pool) + async fn stage_metrics_splits( + &self, + request: StageMetricsSplitsRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let splits_metadata = request.deserialize_splits_metadata()?; + self.stage_parquet_splits_impl(ParquetSplitKind::Metrics, index_uid, splits_metadata) .await - .map_err(|sqlx_error| convert_sqlx_err(&request.index_uid().index_id, sqlx_error))?; + } - info!( - index_uid = %request.index_uid(), - marked_count = marked_split_ids.len(), - "marked metrics splits for deletion" - ); - Ok(EmptyResponse {}) + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn publish_metrics_splits( + &self, + request: PublishMetricsSplitsRequest, + ) -> MetastoreResult { + let checkpoint_delta_opt = request.deserialize_index_checkpoint()?; + let index_uid = request.index_uid().clone(); + self.publish_parquet_splits_impl( + ParquetSplitKind::Metrics, + index_uid, + request.staged_split_ids, + request.replaced_split_ids, + checkpoint_delta_opt, + request.publish_token_opt, + ) + .await + } + + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn list_metrics_splits( + &self, + request: ListMetricsSplitsRequest, + ) -> MetastoreResult { + let query: ListParquetSplitsQuery = serde_utils::from_json_str(&request.query_json)?; + let splits = self + .list_parquet_splits_impl(ParquetSplitKind::Metrics, query) + .await?; + ListMetricsSplitsResponse::try_from_splits(&splits) + } + + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn mark_metrics_splits_for_deletion( + &self, + request: MarkMetricsSplitsForDeletionRequest, + ) -> MetastoreResult { + self.mark_parquet_splits_for_deletion_impl( + ParquetSplitKind::Metrics, + request.index_uid(), + &request.split_ids, + ) + .await } #[instrument(skip_all, fields(index_uid = %request.index_uid()))] @@ -2375,62 +2537,78 @@ impl MetastoreService for PostgresqlMetastore { &self, request: DeleteMetricsSplitsRequest, ) -> MetastoreResult { - if request.split_ids.is_empty() { - return Ok(EmptyResponse {}); - } - - info!( - index_uid = %request.index_uid(), - split_ids = ?request.split_ids, - "deleting metrics splits" - ); - - // Only delete splits that are marked for deletion - const DELETE_SPLITS_QUERY: &str = r#" - DELETE FROM metrics_splits - WHERE - index_uid = $1 - AND split_id = ANY($2) - AND split_state = 'MarkedForDeletion' - RETURNING split_id - "#; + self.delete_parquet_splits_impl( + ParquetSplitKind::Metrics, + request.index_uid(), + &request.split_ids, + ) + .await + } - let deleted_split_ids: Vec = sqlx::query_scalar(DELETE_SPLITS_QUERY) - .bind(request.index_uid()) - .bind(&request.split_ids) - .fetch_all(&self.connection_pool) + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> MetastoreResult { + let index_uid = request.index_uid().clone(); + let splits_metadata = request.deserialize_splits_metadata()?; + self.stage_parquet_splits_impl(ParquetSplitKind::Sketches, index_uid, splits_metadata) .await - .map_err(|sqlx_error| convert_sqlx_err(&request.index_uid().index_id, sqlx_error))?; + } - // Check if any splits could not be deleted - if deleted_split_ids.len() != request.split_ids.len() { - let not_deleted: Vec = request - .split_ids - .iter() - .filter(|id| !deleted_split_ids.contains(id)) - .cloned() - .collect(); + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> MetastoreResult { + let checkpoint_delta_opt = request.deserialize_index_checkpoint()?; + let index_uid = request.index_uid().clone(); + self.publish_parquet_splits_impl( + ParquetSplitKind::Sketches, + index_uid, + request.staged_split_ids, + request.replaced_split_ids, + checkpoint_delta_opt, + request.publish_token_opt, + ) + .await + } - if !not_deleted.is_empty() { - warn!( - index_uid = %request.index_uid(), - not_deleted = ?not_deleted, - "some metrics splits were not in MarkedForDeletion state" - ); - let entity = EntityKind::Splits { - split_ids: not_deleted, - }; - let message = "splits are not marked for deletion".to_string(); - return Err(MetastoreError::FailedPrecondition { entity, message }); - } - } + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> MetastoreResult { + let query: ListParquetSplitsQuery = serde_utils::from_json_str(&request.query_json)?; + let splits = self + .list_parquet_splits_impl(ParquetSplitKind::Sketches, query) + .await?; + ListSketchSplitsResponse::try_from_splits(&splits) + } - info!( - index_uid = %request.index_uid(), - deleted_count = deleted_split_ids.len(), - "deleted metrics splits successfully" - ); - Ok(EmptyResponse {}) + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> MetastoreResult { + self.mark_parquet_splits_for_deletion_impl( + ParquetSplitKind::Sketches, + request.index_uid(), + &request.split_ids, + ) + .await + } + + #[instrument(skip_all, fields(index_uid = %request.index_uid()))] + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> MetastoreResult { + self.delete_parquet_splits_impl( + ParquetSplitKind::Sketches, + request.index_uid(), + &request.split_ids, + ) + .await } } diff --git a/quickwit/quickwit-parquet-engine/src/ingest/arrow_sketches.rs b/quickwit/quickwit-parquet-engine/src/ingest/arrow_sketches.rs new file mode 100644 index 00000000000..0b7b1ed5aed --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/ingest/arrow_sketches.rs @@ -0,0 +1,430 @@ +// 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. + +//! Arrow-based batch building for DDSketch data with dynamic schema discovery. +//! +//! Follows the same two-pass pattern as `ArrowMetricsBatchBuilder`: accumulate +//! data points, discover the union of tag keys, then build the RecordBatch. + +use std::collections::{BTreeSet, HashMap}; +use std::sync::Arc; + +use arrow::array::{ + ArrayRef, Float64Builder, Int16Builder, ListBuilder, RecordBatch, StringDictionaryBuilder, + UInt32Builder, UInt64Builder, +}; +use arrow::datatypes::{DataType, Field, Int32Type, Schema as ArrowSchema}; + +/// A single DDSketch data point with tags. +#[derive(Debug, Clone)] +pub struct SketchDataPoint { + pub metric_name: String, + pub timestamp_secs: u64, + pub count: u64, + pub sum: f64, + pub min: f64, + pub max: f64, + pub flags: u32, + pub keys: Vec, + pub counts: Vec, + pub tags: HashMap, +} + +/// Builder for creating Arrow RecordBatch from SketchDataPoints. +/// +/// Accumulates data points and discovers the schema dynamically at `finish()` +/// time. Uses dictionary encoding for string columns (metric_name, all tags) +/// to achieve significant compression for low cardinality values. +pub struct ArrowSketchBatchBuilder { + data_points: Vec, +} + +impl ArrowSketchBatchBuilder { + /// Creates a new builder with pre-allocated capacity. + pub fn with_capacity(capacity: usize) -> Self { + Self { + data_points: Vec::with_capacity(capacity), + } + } + + /// Appends a SketchDataPoint to the batch. + pub fn append(&mut self, data_point: SketchDataPoint) { + self.data_points.push(data_point); + } + + /// Finalizes and returns the RecordBatch. + /// + /// Performs two passes: + /// 1. Schema discovery: scans all data points to collect the union of tag keys. + /// 2. Array building: creates per-column builders and populates them. + pub fn finish(self) -> RecordBatch { + let num_rows = self.data_points.len(); + + // Pass 1: discover all tag keys across all data points. + let mut tag_keys: BTreeSet<&str> = BTreeSet::new(); + for dp in &self.data_points { + for key in dp.tags.keys() { + tag_keys.insert(key.as_str()); + } + } + let sorted_tag_keys: Vec = tag_keys.into_iter().map(str::to_owned).collect(); + + // Build the Arrow schema dynamically + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let mut fields = Vec::with_capacity(9 + sorted_tag_keys.len()); + fields.push(Field::new("metric_name", dict_type.clone(), false)); + fields.push(Field::new("timestamp_secs", DataType::UInt64, false)); + fields.push(Field::new("count", DataType::UInt64, false)); + fields.push(Field::new("sum", DataType::Float64, false)); + fields.push(Field::new("min", DataType::Float64, false)); + fields.push(Field::new("max", DataType::Float64, false)); + fields.push(Field::new("flags", DataType::UInt32, false)); + fields.push(Field::new( + "keys", + DataType::List(Arc::new(Field::new("item", DataType::Int16, false))), + false, + )); + fields.push(Field::new( + "counts", + DataType::List(Arc::new(Field::new("item", DataType::UInt64, false))), + false, + )); + + for tag_key in &sorted_tag_keys { + fields.push(Field::new(tag_key, dict_type.clone(), true)); + } + + let schema = Arc::new(ArrowSchema::new(fields)); + + // Pass 2: build arrays + let mut metric_name_builder: StringDictionaryBuilder = + StringDictionaryBuilder::new(); + let mut timestamp_secs_builder = UInt64Builder::with_capacity(num_rows); + let mut count_builder = UInt64Builder::with_capacity(num_rows); + let mut sum_builder = Float64Builder::with_capacity(num_rows); + let mut min_builder = Float64Builder::with_capacity(num_rows); + let mut max_builder = Float64Builder::with_capacity(num_rows); + let mut flags_builder = UInt32Builder::with_capacity(num_rows); + let mut keys_builder = ListBuilder::new(Int16Builder::new()).with_field(Field::new( + "item", + DataType::Int16, + false, + )); + let mut counts_builder = ListBuilder::new(UInt64Builder::new()).with_field(Field::new( + "item", + DataType::UInt64, + false, + )); + + let mut tag_builders: Vec> = sorted_tag_keys + .iter() + .map(|_| StringDictionaryBuilder::new()) + .collect(); + + for dp in &self.data_points { + metric_name_builder.append_value(&dp.metric_name); + timestamp_secs_builder.append_value(dp.timestamp_secs); + count_builder.append_value(dp.count); + sum_builder.append_value(dp.sum); + min_builder.append_value(dp.min); + max_builder.append_value(dp.max); + flags_builder.append_value(dp.flags); + + let keys_inner = keys_builder.values(); + for &k in &dp.keys { + keys_inner.append_value(k); + } + keys_builder.append(true); + + let counts_inner = counts_builder.values(); + for &c in &dp.counts { + counts_inner.append_value(c); + } + counts_builder.append(true); + + for (tag_idx, tag_key) in sorted_tag_keys.iter().enumerate() { + match dp.tags.get(tag_key) { + Some(tag_val) => tag_builders[tag_idx].append_value(tag_val), + None => tag_builders[tag_idx].append_null(), + } + } + } + + let mut arrays: Vec = Vec::with_capacity(9 + sorted_tag_keys.len()); + arrays.push(Arc::new(metric_name_builder.finish())); + arrays.push(Arc::new(timestamp_secs_builder.finish())); + arrays.push(Arc::new(count_builder.finish())); + arrays.push(Arc::new(sum_builder.finish())); + arrays.push(Arc::new(min_builder.finish())); + arrays.push(Arc::new(max_builder.finish())); + arrays.push(Arc::new(flags_builder.finish())); + arrays.push(Arc::new(keys_builder.finish())); + arrays.push(Arc::new(counts_builder.finish())); + + for tag_builder in &mut tag_builders { + arrays.push(Arc::new(tag_builder.finish())); + } + + RecordBatch::try_new(schema, arrays).expect( + "all arrays were built from the same schema discovered above; schema mismatch is a \ + bug in ArrowSketchBatchBuilder", + ) + } + + /// Returns the number of rows appended so far. + pub fn len(&self) -> usize { + self.data_points.len() + } + + /// Returns true if no rows have been appended. + pub fn is_empty(&self) -> bool { + self.data_points.is_empty() + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + + use super::*; + + fn make_test_sketch_point() -> SketchDataPoint { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), "api".to_string()); + tags.insert("env".to_string(), "prod".to_string()); + tags.insert("host".to_string(), "host-001".to_string()); + + SketchDataPoint { + metric_name: "req.latency".to_string(), + timestamp_secs: 1704067200, + count: 100, + sum: 5000.0, + min: 1.0, + max: 200.0, + flags: 0, + keys: vec![100, 200, 300], + counts: vec![50, 30, 20], + tags, + } + } + + #[test] + fn test_sketch_batch_builder_single_row() { + let dp = make_test_sketch_point(); + let mut builder = ArrowSketchBatchBuilder::with_capacity(1); + builder.append(dp); + + assert_eq!(builder.len(), 1); + assert!(!builder.is_empty()); + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 1); + // 9 fixed columns + 3 tag columns (env, host, service) + assert_eq!(batch.num_columns(), 12); + + // Verify schema field names + let schema = batch.schema(); + let field_names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!( + field_names, + vec![ + "metric_name", + "timestamp_secs", + "count", + "sum", + "min", + "max", + "flags", + "keys", + "counts", + "env", + "host", + "service", + ] + ); + } + + #[test] + fn test_sketch_batch_builder_multiple_rows() { + let mut builder = ArrowSketchBatchBuilder::with_capacity(100); + + for idx in 0..100 { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), format!("svc-{}", idx % 10)); + tags.insert("env".to_string(), "prod".to_string()); + + let dp = SketchDataPoint { + metric_name: "test.latency".to_string(), + timestamp_secs: 1704067200 + idx as u64, + count: 50, + sum: 1000.0, + min: 0.5, + max: 100.0, + flags: 0, + keys: vec![100, 200], + counts: vec![30, 20], + tags, + }; + builder.append(dp); + } + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 100); + // 9 fixed + 2 tags + assert_eq!(batch.num_columns(), 11); + } + + #[test] + fn test_sketch_batch_dynamic_schema_discovery() { + let mut builder = ArrowSketchBatchBuilder::with_capacity(2); + + // First point has tags: env, host + let mut tags1 = HashMap::new(); + tags1.insert("env".to_string(), "prod".to_string()); + tags1.insert("host".to_string(), "host-1".to_string()); + builder.append(SketchDataPoint { + metric_name: "m1".to_string(), + timestamp_secs: 1000, + count: 10, + sum: 100.0, + min: 1.0, + max: 50.0, + flags: 0, + keys: vec![100], + counts: vec![10], + tags: tags1, + }); + + // Second point has tags: env, region (different set) + let mut tags2 = HashMap::new(); + tags2.insert("env".to_string(), "staging".to_string()); + tags2.insert("region".to_string(), "us-west".to_string()); + builder.append(SketchDataPoint { + metric_name: "m2".to_string(), + timestamp_secs: 1001, + count: 20, + sum: 200.0, + min: 2.0, + max: 80.0, + flags: 0, + keys: vec![200], + counts: vec![20], + tags: tags2, + }); + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 2); + // 9 fixed + 3 tags (env, host, region) + assert_eq!(batch.num_columns(), 12); + + let schema = batch.schema(); + let tag_names: Vec<&str> = schema + .fields() + .iter() + .skip(9) + .map(|f| f.name().as_str()) + .collect(); + assert_eq!(tag_names, vec!["env", "host", "region"]); + } + + #[test] + fn test_sketch_batch_empty() { + let builder = ArrowSketchBatchBuilder::with_capacity(0); + assert!(builder.is_empty()); + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 0); + // 9 fixed columns, no tags + assert_eq!(batch.num_columns(), 9); + } + + #[test] + fn test_sketch_batch_list_columns() { + let mut builder = ArrowSketchBatchBuilder::with_capacity(2); + + builder.append(SketchDataPoint { + metric_name: "m".to_string(), + timestamp_secs: 1000, + count: 10, + sum: 100.0, + min: 1.0, + max: 50.0, + flags: 0, + keys: vec![100, 200, 300], + counts: vec![5, 3, 2], + tags: HashMap::new(), + }); + + // Different length lists + builder.append(SketchDataPoint { + metric_name: "m".to_string(), + timestamp_secs: 1001, + count: 5, + sum: 50.0, + min: 2.0, + max: 30.0, + flags: 0, + keys: vec![150], + counts: vec![5], + tags: HashMap::new(), + }); + + let batch = builder.finish(); + assert_eq!(batch.num_rows(), 2); + + // Verify keys column is List type + let schema = batch.schema(); + let keys_field = schema.field_with_name("keys").unwrap(); + assert!(matches!(keys_field.data_type(), DataType::List(_))); + + // Verify counts column is List type + let counts_field = schema.field_with_name("counts").unwrap(); + assert!(matches!(counts_field.data_type(), DataType::List(_))); + } + + #[test] + fn test_sketch_batch_ipc_round_trip() { + use super::super::processor::{ipc_to_record_batch, record_batch_to_ipc}; + + let mut builder = ArrowSketchBatchBuilder::with_capacity(5); + for idx in 0..5 { + let mut tags = HashMap::new(); + tags.insert("service".to_string(), format!("svc-{}", idx % 3)); + + builder.append(SketchDataPoint { + metric_name: "test.sketch".to_string(), + timestamp_secs: 1000 + idx as u64, + count: 50, + sum: 500.0, + min: 1.0, + max: 100.0, + flags: 0, + keys: vec![100, 200], + counts: vec![30, 20], + tags, + }); + } + + let original_batch = builder.finish(); + + let ipc_bytes = record_batch_to_ipc(&original_batch).unwrap(); + assert!(!ipc_bytes.is_empty()); + + let recovered_batch = ipc_to_record_batch(&ipc_bytes).unwrap(); + + assert_eq!(recovered_batch.num_rows(), original_batch.num_rows()); + assert_eq!(recovered_batch.num_columns(), original_batch.num_columns()); + assert_eq!(recovered_batch.schema(), original_batch.schema()); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/ingest/mod.rs b/quickwit/quickwit-parquet-engine/src/ingest/mod.rs index 41604a6d32f..11f6284b534 100644 --- a/quickwit/quickwit-parquet-engine/src/ingest/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/ingest/mod.rs @@ -17,6 +17,10 @@ //! Provides high-throughput metrics ingestion by converting Arrow IPC batches //! directly to Parquet splits, bypassing Tantivy entirely. +pub mod arrow_sketches; pub mod processor; +pub mod sketch_processor; +pub use arrow_sketches::{ArrowSketchBatchBuilder, SketchDataPoint}; pub use processor::{IngestError, ParquetIngestProcessor, record_batch_to_ipc}; +pub use sketch_processor::SketchParquetIngestProcessor; diff --git a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs index 21412414b2b..65cdc2fac1d 100644 --- a/quickwit/quickwit-parquet-engine/src/ingest/processor.rs +++ b/quickwit/quickwit-parquet-engine/src/ingest/processor.rs @@ -65,6 +65,7 @@ impl ParquetIngestProcessor { // Record bytes ingested PARQUET_ENGINE_METRICS .ingest_bytes_total + .with_label_values(["points"]) .inc_by(ipc_bytes.len() as u64); let batch = match ipc_to_record_batch(ipc_bytes) { @@ -72,7 +73,7 @@ impl ParquetIngestProcessor { Err(e) => { PARQUET_ENGINE_METRICS .errors_total - .with_label_values(["ingest"]) + .with_label_values(["ingest", "points"]) .inc(); return Err(e); } @@ -81,7 +82,7 @@ impl ParquetIngestProcessor { if let Err(e) = self.validate_schema(&batch) { PARQUET_ENGINE_METRICS .errors_total - .with_label_values(["ingest"]) + .with_label_values(["ingest", "points"]) .inc(); return Err(e); } @@ -101,7 +102,7 @@ impl ParquetIngestProcessor { } /// Deserialize Arrow IPC stream format to a RecordBatch. -fn ipc_to_record_batch(ipc_bytes: &[u8]) -> Result { +pub(crate) fn ipc_to_record_batch(ipc_bytes: &[u8]) -> Result { let cursor = Cursor::new(ipc_bytes); let reader = StreamReader::try_new(cursor, None)?; diff --git a/quickwit/quickwit-parquet-engine/src/ingest/sketch_processor.rs b/quickwit/quickwit-parquet-engine/src/ingest/sketch_processor.rs new file mode 100644 index 00000000000..3a5cb3b49d0 --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/ingest/sketch_processor.rs @@ -0,0 +1,350 @@ +// 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. + +//! DDSketch ingest processor for Arrow IPC to RecordBatch conversion. + +use arrow::array::AsArray; +use arrow::record_batch::RecordBatch; +use tracing::{debug, instrument, warn}; + +use super::processor::IngestError; +use crate::metrics::PARQUET_ENGINE_METRICS; +use crate::schema::validate_required_sketch_fields; + +/// Processor that converts Arrow IPC bytes to RecordBatch for DDSketch data. +/// +/// Validates required sketch fields and DDSketch-specific invariants: +/// - keys and counts arrays must have the same length per row +#[derive(Default)] +pub struct SketchParquetIngestProcessor; + +impl SketchParquetIngestProcessor { + /// Create a new SketchParquetIngestProcessor. + pub fn new() -> Self { + Self + } + + /// Convert Arrow IPC bytes to RecordBatch. + /// + /// Returns error if IPC is malformed, schema doesn't match, or + /// sketch arrays are inconsistent. + #[instrument(skip(self, ipc_bytes), fields(bytes_len = ipc_bytes.len()))] + pub fn process_ipc(&self, ipc_bytes: &[u8]) -> Result { + PARQUET_ENGINE_METRICS + .ingest_bytes_total + .with_label_values(["sketches"]) + .inc_by(ipc_bytes.len() as u64); + + let batch = match super::processor::ipc_to_record_batch(ipc_bytes) { + Ok(batch) => batch, + Err(err) => { + PARQUET_ENGINE_METRICS + .errors_total + .with_label_values(["ingest", "sketches"]) + .inc(); + return Err(err); + } + }; + + if let Err(err) = self.validate_schema(&batch) { + PARQUET_ENGINE_METRICS + .errors_total + .with_label_values(["ingest", "sketches"]) + .inc(); + return Err(err); + } + + if let Err(err) = self.validate_sketch_arrays(&batch) { + PARQUET_ENGINE_METRICS + .errors_total + .with_label_values(["ingest", "sketches"]) + .inc(); + return Err(err); + } + + debug!( + num_rows = batch.num_rows(), + "successfully decoded sketch IPC to RecordBatch" + ); + Ok(batch) + } + + /// Validate that the RecordBatch schema contains all required sketch fields. + fn validate_schema(&self, batch: &RecordBatch) -> Result<(), IngestError> { + validate_required_sketch_fields(batch.schema().as_ref()) + .map_err(|err| IngestError::SchemaValidation(err.to_string())) + } + + /// Validate DDSketch-specific array invariants. + fn validate_sketch_arrays(&self, batch: &RecordBatch) -> Result<(), IngestError> { + let keys_idx = batch.schema().index_of("keys").map_err(|_| { + IngestError::SchemaValidation("missing required sketch field 'keys'".to_string()) + })?; + let counts_idx = batch.schema().index_of("counts").map_err(|_| { + IngestError::SchemaValidation("missing required sketch field 'counts'".to_string()) + })?; + + let keys_col = batch.column(keys_idx); + let counts_col = batch.column(counts_idx); + + let keys_list = keys_col.as_list::(); + let counts_list = counts_col.as_list::(); + + for row in 0..batch.num_rows() { + let keys_len = keys_list.value(row).len(); + let counts_len = counts_list.value(row).len(); + if keys_len != counts_len { + warn!( + row, + keys_len, + counts_len, + "sketch array length mismatch: keys and counts must have same length" + ); + return Err(IngestError::SchemaValidation(format!( + "sketch array length mismatch at row {}: keys has {} elements but counts has \ + {}", + row, keys_len, counts_len, + ))); + } + } + Ok(()) + } +} + +#[cfg(test)] +pub(crate) mod tests { + use std::sync::Arc; + + use arrow::array::{ + ArrayRef, Float64Array, Int16Array, ListArray, UInt32Array, UInt64Array, + }; + use arrow::buffer::OffsetBuffer; + use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; + + use super::*; + use crate::ingest::processor::record_batch_to_ipc; + use crate::test_helpers::{create_dict_array, create_nullable_dict_array}; + + /// Build a List array from slices. + fn create_int16_list_array(rows: &[&[i16]]) -> ArrayRef { + let mut offsets = vec![0i32]; + let mut values = Vec::new(); + for row in rows { + values.extend_from_slice(row); + offsets.push(values.len() as i32); + } + let values_array = Int16Array::from(values); + let offsets = OffsetBuffer::new(offsets.into()); + let field = Arc::new(Field::new("item", DataType::Int16, false)); + Arc::new(ListArray::new(field, offsets, Arc::new(values_array), None)) + } + + /// Build a List array from slices. + fn create_uint64_list_array(rows: &[&[u64]]) -> ArrayRef { + let mut offsets = vec![0i32]; + let mut values = Vec::new(); + for row in rows { + values.extend_from_slice(row); + offsets.push(values.len() as i32); + } + let values_array = UInt64Array::from(values); + let offsets = OffsetBuffer::new(offsets.into()); + let field = Arc::new(Field::new("item", DataType::UInt64, false)); + Arc::new(ListArray::new(field, offsets, Arc::new(values_array), None)) + } + + /// Create a test sketch batch with the 9 required fields plus the specified + /// nullable dictionary-encoded tag columns. + pub(crate) fn create_test_sketch_batch_with_tags( + num_rows: usize, + tags: &[(&str, &str)], + ) -> RecordBatch { + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + + let mut fields = vec![ + Field::new("metric_name", dict_type.clone(), false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("count", DataType::UInt64, false), + Field::new("sum", DataType::Float64, false), + Field::new("min", DataType::Float64, false), + Field::new("max", DataType::Float64, false), + Field::new("flags", DataType::UInt32, false), + Field::new( + "keys", + DataType::List(Arc::new(Field::new("item", DataType::Int16, false))), + false, + ), + Field::new( + "counts", + DataType::List(Arc::new(Field::new("item", DataType::UInt64, false))), + false, + ), + ]; + for (tag_name, _) in tags { + fields.push(Field::new(*tag_name, dict_type.clone(), true)); + } + let schema = Arc::new(ArrowSchema::new(fields)); + + let metric_name: ArrayRef = create_dict_array(&vec!["req.latency"; num_rows]); + let timestamps: Vec = (0..num_rows).map(|i| 1000 + i as u64 * 10).collect(); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(timestamps)); + let count: ArrayRef = Arc::new(UInt64Array::from(vec![100u64; num_rows])); + let sum: ArrayRef = Arc::new(Float64Array::from(vec![50.0; num_rows])); + let min_arr: ArrayRef = Arc::new(Float64Array::from(vec![1.0; num_rows])); + let max_arr: ArrayRef = Arc::new(Float64Array::from(vec![100.0; num_rows])); + let flags: ArrayRef = Arc::new(UInt32Array::from(vec![0u32; num_rows])); + + let keys_rows: Vec<&[i16]> = vec![&[100, 200, 300]; num_rows]; + let keys: ArrayRef = create_int16_list_array(&keys_rows); + + let counts_rows: Vec<&[u64]> = vec![&[50, 30, 20]; num_rows]; + let counts: ArrayRef = create_uint64_list_array(&counts_rows); + + let mut columns: Vec = vec![ + metric_name, + timestamp_secs, + count, + sum, + min_arr, + max_arr, + flags, + keys, + counts, + ]; + + for (_, tag_value) in tags { + let tag_values: Vec> = vec![Some(tag_value); num_rows]; + columns.push(create_nullable_dict_array(&tag_values)); + } + + RecordBatch::try_new(schema, columns).unwrap() + } + + /// Create a test sketch batch with default tags (service, host). + pub(crate) fn create_test_sketch_batch(num_rows: usize) -> RecordBatch { + create_test_sketch_batch_with_tags(num_rows, &[("service", "api"), ("host", "host-001")]) + } + + #[test] + fn test_process_valid_sketch_ipc() { + let processor = SketchParquetIngestProcessor::new(); + let batch = create_test_sketch_batch(10); + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let result = processor.process_ipc(&ipc_bytes); + assert!(result.is_ok()); + assert_eq!(result.unwrap().num_rows(), 10); + } + + #[test] + fn test_reject_wrong_schema() { + let processor = SketchParquetIngestProcessor::new(); + + // Create a batch missing required sketch fields (only has metric_name and timestamp) + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type, false), + Field::new("timestamp_secs", DataType::UInt64, false), + ])); + let batch = RecordBatch::try_new( + schema, + vec![ + create_dict_array(&["cpu.usage"]), + Arc::new(UInt64Array::from(vec![100u64])) as ArrayRef, + ], + ) + .unwrap(); + + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let result = processor.process_ipc(&ipc_bytes); + assert!(result.is_err()); + } + + #[test] + fn test_reject_mismatched_keys_counts_lengths() { + let processor = SketchParquetIngestProcessor::new(); + + // Build a batch with keys=[1,2] but counts=[10] (length mismatch) + let dict_type = DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)); + let schema = Arc::new(ArrowSchema::new(vec![ + Field::new("metric_name", dict_type, false), + Field::new("timestamp_secs", DataType::UInt64, false), + Field::new("count", DataType::UInt64, false), + Field::new("sum", DataType::Float64, false), + Field::new("min", DataType::Float64, false), + Field::new("max", DataType::Float64, false), + Field::new("flags", DataType::UInt32, false), + Field::new( + "keys", + DataType::List(Arc::new(Field::new("item", DataType::Int16, false))), + false, + ), + Field::new( + "counts", + DataType::List(Arc::new(Field::new("item", DataType::UInt64, false))), + false, + ), + ])); + + let metric_name: ArrayRef = create_dict_array(&["req.latency"]); + let timestamp_secs: ArrayRef = Arc::new(UInt64Array::from(vec![1000u64])); + let count: ArrayRef = Arc::new(UInt64Array::from(vec![100u64])); + let sum: ArrayRef = Arc::new(Float64Array::from(vec![50.0])); + let min_arr: ArrayRef = Arc::new(Float64Array::from(vec![1.0])); + let max_arr: ArrayRef = Arc::new(Float64Array::from(vec![100.0])); + let flags: ArrayRef = Arc::new(UInt32Array::from(vec![0u32])); + + // keys has 2 elements, counts has 1 element (mismatch!) + let keys: ArrayRef = create_int16_list_array(&[&[1i16, 2]]); + let counts: ArrayRef = create_uint64_list_array(&[&[10u64]]); + + let batch = RecordBatch::try_new( + schema, + vec![ + metric_name, + timestamp_secs, + count, + sum, + min_arr, + max_arr, + flags, + keys, + counts, + ], + ) + .unwrap(); + + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let result = processor.process_ipc(&ipc_bytes); + assert!(result.is_err()); + } + + #[test] + fn test_accept_sketch_with_extra_tag_columns() { + let processor = SketchParquetIngestProcessor::new(); + + // Sketch batch with extra tag columns should be accepted + let batch = create_test_sketch_batch_with_tags( + 3, + &[ + ("service", "api"), + ("host", "host-001"), + ("region", "us-east-1"), + ], + ); + let ipc_bytes = record_batch_to_ipc(&batch).unwrap(); + let result = processor.process_ipc(&ipc_bytes); + assert!(result.is_ok()); + assert_eq!(result.unwrap().num_rows(), 3); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/metrics.rs b/quickwit/quickwit-parquet-engine/src/metrics.rs index de55f72deab..64ff447d851 100644 --- a/quickwit/quickwit-parquet-engine/src/metrics.rs +++ b/quickwit/quickwit-parquet-engine/src/metrics.rs @@ -41,8 +41,9 @@ pub struct ParquetEngineMetrics { pub index_batches_total: IntCounter, /// Total number of rows accumulated during indexing. pub index_rows_total: IntCounter, - /// Total number of bytes received from IPC payloads during ingestion. - pub ingest_bytes_total: IntCounter, + /// Total number of bytes received from IPC payloads during ingestion, by kind + /// (points/sketches). + pub ingest_bytes_total: IntCounterVec<1>, /// Histogram of add_batch durations (seconds), including any triggered flush. pub index_batch_duration_seconds: Histogram, /// Total number of splits written to storage. @@ -53,8 +54,8 @@ pub struct ParquetEngineMetrics { pub query_duration_seconds: Histogram, /// Total number of rows returned from queries. pub query_rows_returned: IntCounter, - /// Errors by operation type: ingest, query, storage. - pub errors_total: IntCounterVec<1>, + /// Errors by operation type and kind (points/sketches). + pub errors_total: IntCounterVec<2>, } impl Default for ParquetEngineMetrics { @@ -72,11 +73,12 @@ impl Default for ParquetEngineMetrics { SUBSYSTEM, &[], ), - ingest_bytes_total: new_counter( + ingest_bytes_total: new_counter_vec( "ingest_bytes_total", "Total number of bytes received from IPC payloads during ingestion.", SUBSYSTEM, &[], + ["kind"], ), index_batch_duration_seconds: new_histogram( "index_batch_duration_seconds", @@ -110,10 +112,10 @@ impl Default for ParquetEngineMetrics { ), errors_total: new_counter_vec( "errors_total", - "Total errors by operation type.", + "Total errors by operation type and kind.", SUBSYSTEM, &[], - ["operation"], + ["operation", "kind"], ), } } diff --git a/quickwit/quickwit-parquet-engine/src/schema/fields.rs b/quickwit/quickwit-parquet-engine/src/schema/fields.rs index 9f46dcf3b8c..73291d8e91e 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/fields.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/fields.rs @@ -14,13 +14,28 @@ //! Parquet field definitions with sort order constants and validation. +use std::sync::Arc; + use anyhow::{Result, bail}; -use arrow::datatypes::DataType; +use arrow::datatypes::{DataType, Field}; -/// Required field names that must exist in every batch. +/// Required field names that must exist in every metrics batch. pub const REQUIRED_FIELDS: &[&str] = &["metric_name", "metric_type", "timestamp_secs", "value"]; -/// Sort order column names. Columns not present in a batch are skipped. +/// Required field names that must exist in every sketch batch. +pub const SKETCH_REQUIRED_FIELDS: &[&str] = &[ + "metric_name", + "timestamp_secs", + "count", + "sum", + "min", + "max", + "flags", + "keys", + "counts", +]; + +/// Sort order column names for metrics. Columns not present in a batch are skipped. pub const SORT_ORDER: &[&str] = &[ "metric_name", "service", @@ -31,6 +46,17 @@ pub const SORT_ORDER: &[&str] = &[ "timestamp_secs", ]; +/// Sort order column names for sketches. Same tag columns as metrics. +pub const SKETCH_SORT_ORDER: &[&str] = &[ + "metric_name", + "service", + "env", + "datacenter", + "region", + "host", + "timestamp_secs", +]; + /// Arrow type for required fields by name. pub fn required_field_type(name: &str) -> Option { match name { @@ -66,3 +92,50 @@ pub fn validate_required_fields(schema: &arrow::datatypes::Schema) -> Result<()> } Ok(()) } + +/// Arrow type for required sketch fields by name. +pub fn sketch_required_field_type(name: &str) -> Option { + match name { + "metric_name" => Some(DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + )), + "timestamp_secs" => Some(DataType::UInt64), + "count" => Some(DataType::UInt64), + "sum" | "min" | "max" => Some(DataType::Float64), + "flags" => Some(DataType::UInt32), + "keys" => Some(DataType::List(Arc::new(Field::new( + "item", + DataType::Int16, + false, + )))), + "counts" => Some(DataType::List(Arc::new(Field::new( + "item", + DataType::UInt64, + false, + )))), + _ => None, + } +} + +/// Validate that a batch schema contains all required sketch fields with correct types. +pub fn validate_required_sketch_fields(schema: &arrow::datatypes::Schema) -> Result<()> { + for &name in SKETCH_REQUIRED_FIELDS { + match schema.index_of(name) { + Ok(idx) => { + let expected_type = sketch_required_field_type(name).unwrap(); + let actual_type = schema.field(idx).data_type(); + if *actual_type != expected_type { + bail!( + "field '{}' has type {:?}, expected {:?}", + name, + actual_type, + expected_type + ); + } + } + Err(_) => bail!("missing required sketch field '{}'", name), + } + } + Ok(()) +} diff --git a/quickwit/quickwit-parquet-engine/src/schema/mod.rs b/quickwit/quickwit-parquet-engine/src/schema/mod.rs index f9b5c06d9c4..402a2651475 100644 --- a/quickwit/quickwit-parquet-engine/src/schema/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/schema/mod.rs @@ -19,6 +19,13 @@ mod fields; mod parquet; +pub mod sketch_fields; +pub mod sketch_schema; -pub use fields::{REQUIRED_FIELDS, SORT_ORDER, required_field_type, validate_required_fields}; +pub use fields::{ + REQUIRED_FIELDS, SKETCH_REQUIRED_FIELDS, SKETCH_SORT_ORDER, SORT_ORDER, required_field_type, + sketch_required_field_type, validate_required_fields, validate_required_sketch_fields, +}; pub use parquet::ParquetSchema; +pub use sketch_fields::SketchParquetField; +pub use sketch_schema::SketchParquetSchema; diff --git a/quickwit/quickwit-parquet-engine/src/schema/sketch_fields.rs b/quickwit/quickwit-parquet-engine/src/schema/sketch_fields.rs new file mode 100644 index 00000000000..f2e4ab6df0c --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/schema/sketch_fields.rs @@ -0,0 +1,140 @@ +// 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. + +//! DDSketch parquet required field definitions. +//! +//! Tag columns are dynamic (each unique tag key becomes its own column), +//! so only the required sketch-specific fields are defined here. + +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field}; + +/// Required fields in the DDSketch parquet schema. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] +pub enum SketchParquetField { + MetricName, + TimestampSecs, + Count, + Sum, + Min, + Max, + Flags, + Keys, + Counts, +} + +impl SketchParquetField { + /// Field name as stored in Parquet. + pub fn name(&self) -> &'static str { + match self { + Self::MetricName => "metric_name", + Self::TimestampSecs => "timestamp_secs", + Self::Count => "count", + Self::Sum => "sum", + Self::Min => "min", + Self::Max => "max", + Self::Flags => "flags", + Self::Keys => "keys", + Self::Counts => "counts", + } + } + + /// Whether this field is nullable. + pub fn is_nullable(&self) -> bool { + false + } + + /// Arrow DataType for this field. + pub fn arrow_type(&self) -> DataType { + match self { + Self::MetricName => { + DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)) + } + Self::TimestampSecs => DataType::UInt64, + Self::Count => DataType::UInt64, + Self::Sum | Self::Min | Self::Max => DataType::Float64, + Self::Flags => DataType::UInt32, + Self::Keys => DataType::List(Arc::new(Field::new("item", DataType::Int16, false))), + Self::Counts => DataType::List(Arc::new(Field::new("item", DataType::UInt64, false))), + } + } + + /// Convert to Arrow Field. + pub fn to_arrow_field(&self) -> Field { + Field::new(self.name(), self.arrow_type(), self.is_nullable()) + } + + /// All required fields in schema order. + pub fn all() -> &'static [SketchParquetField] { + &[ + Self::MetricName, + Self::TimestampSecs, + Self::Count, + Self::Sum, + Self::Min, + Self::Max, + Self::Flags, + Self::Keys, + Self::Counts, + ] + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_sketch_field_count() { + assert_eq!(SketchParquetField::all().len(), 9); + } + + #[test] + fn test_sketch_field_names() { + assert_eq!(SketchParquetField::MetricName.name(), "metric_name"); + assert_eq!(SketchParquetField::Keys.name(), "keys"); + assert_eq!(SketchParquetField::Counts.name(), "counts"); + assert_eq!(SketchParquetField::Flags.name(), "flags"); + assert_eq!(SketchParquetField::Count.name(), "count"); + assert_eq!(SketchParquetField::Sum.name(), "sum"); + assert_eq!(SketchParquetField::Min.name(), "min"); + assert_eq!(SketchParquetField::Max.name(), "max"); + } + + #[test] + fn test_sketch_field_nullability() { + assert!(!SketchParquetField::MetricName.is_nullable()); + assert!(!SketchParquetField::TimestampSecs.is_nullable()); + assert!(!SketchParquetField::Count.is_nullable()); + assert!(!SketchParquetField::Keys.is_nullable()); + assert!(!SketchParquetField::Counts.is_nullable()); + } + + #[test] + fn test_sketch_field_arrow_types() { + use arrow::datatypes::DataType; + assert!(matches!( + SketchParquetField::Keys.arrow_type(), + DataType::List(_) + )); + assert!(matches!( + SketchParquetField::Counts.arrow_type(), + DataType::List(_) + )); + assert_eq!(SketchParquetField::Count.arrow_type(), DataType::UInt64); + assert_eq!(SketchParquetField::Sum.arrow_type(), DataType::Float64); + assert_eq!(SketchParquetField::Flags.arrow_type(), DataType::UInt32); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/schema/sketch_schema.rs b/quickwit/quickwit-parquet-engine/src/schema/sketch_schema.rs new file mode 100644 index 00000000000..d65022eb4ef --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/schema/sketch_schema.rs @@ -0,0 +1,91 @@ +// 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. + +//! DDSketch parquet schema construction. + +use std::sync::Arc; + +use arrow::datatypes::{Schema as ArrowSchema, SchemaRef}; + +use super::sketch_fields::SketchParquetField; + +/// Parquet schema for DDSketch storage. +#[derive(Debug, Clone)] +pub struct SketchParquetSchema { + arrow_schema: SchemaRef, +} + +impl SketchParquetSchema { + /// Returns the 9-field required-only schema. + /// + /// The pipeline now writes dynamic schemas derived from the data (tag + /// columns are discovered at runtime); prefer `from_arrow_schema` when you + /// have a batch schema available. + pub fn new() -> Self { + let fields: Vec<_> = SketchParquetField::all() + .iter() + .map(|f| f.to_arrow_field()) + .collect(); + let arrow_schema = Arc::new(ArrowSchema::new(fields)); + Self { arrow_schema } + } + + /// Create a SketchParquetSchema from an existing Arrow schema. + pub fn from_arrow_schema(schema: SchemaRef) -> Self { + Self { + arrow_schema: schema, + } + } + + /// Get the Arrow schema. + pub fn arrow_schema(&self) -> &SchemaRef { + &self.arrow_schema + } + + /// Number of fields in schema. + pub fn num_fields(&self) -> usize { + self.arrow_schema.fields().len() + } + + /// Get field by name. + pub fn field(&self, name: &str) -> Option<&arrow::datatypes::Field> { + self.arrow_schema.field_with_name(name).ok() + } +} + +impl Default for SketchParquetSchema { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_sketch_schema_creation() { + let schema = SketchParquetSchema::new(); + assert_eq!(schema.num_fields(), 9); + } + + #[test] + fn test_sketch_field_lookup() { + let schema = SketchParquetSchema::new(); + assert!(schema.field("keys").is_some()); + assert!(schema.field("counts").is_some()); + assert!(schema.field("metric_name").is_some()); + assert!(schema.field("nonexistent").is_none()); + } +} diff --git a/quickwit/quickwit-parquet-engine/src/split/format.rs b/quickwit/quickwit-parquet-engine/src/split/format.rs index d5722623815..6a9d7a828d9 100644 --- a/quickwit/quickwit-parquet-engine/src/split/format.rs +++ b/quickwit/quickwit-parquet-engine/src/split/format.rs @@ -16,7 +16,7 @@ use serde::{Deserialize, Serialize}; -use super::metadata::{MetricsSplitMetadata, SplitId, TimeRange}; +use super::metadata::{ParquetSplitId, ParquetSplitMetadata, TimeRange}; /// A parquet split - the storage unit for metrics data. /// @@ -25,7 +25,7 @@ use super::metadata::{MetricsSplitMetadata, SplitId, TimeRange}; #[derive(Debug, Clone, Serialize, Deserialize)] pub struct ParquetSplit { /// Split metadata. - pub metadata: MetricsSplitMetadata, + pub metadata: ParquetSplitMetadata, /// Format version for forward compatibility. pub format_version: u32, @@ -36,7 +36,7 @@ pub const CURRENT_FORMAT_VERSION: u32 = 1; impl ParquetSplit { /// Create a new ParquetSplit. - pub fn new(metadata: MetricsSplitMetadata) -> Self { + pub fn new(metadata: ParquetSplitMetadata) -> Self { Self { metadata, format_version: CURRENT_FORMAT_VERSION, @@ -44,7 +44,7 @@ impl ParquetSplit { } /// Get the split ID. - pub fn id(&self) -> &SplitId { + pub fn id(&self) -> &ParquetSplitId { &self.metadata.split_id } @@ -79,7 +79,7 @@ mod tests { use super::*; fn create_test_split() -> ParquetSplit { - let metadata = MetricsSplitMetadata::builder() + let metadata = ParquetSplitMetadata::builder() .index_uid("test-index:00000000000000000000000000") .time_range(TimeRange::new(1000, 2000)) .num_rows(10000) diff --git a/quickwit/quickwit-parquet-engine/src/split/metadata.rs b/quickwit/quickwit-parquet-engine/src/split/metadata.rs index 992775be862..fe997835ed6 100644 --- a/quickwit/quickwit-parquet-engine/src/split/metadata.rs +++ b/quickwit/quickwit-parquet-engine/src/split/metadata.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Metrics split metadata definitions. +//! Unified parquet split metadata definitions for both metrics and sketch splits. use std::collections::{HashMap, HashSet}; use std::time::SystemTime; @@ -26,20 +26,66 @@ pub const TAG_DATACENTER: &str = "datacenter"; pub const TAG_REGION: &str = "region"; pub const TAG_HOST: &str = "host"; -/// Unique identifier for a metrics split. +/// Distinguishes between metrics and sketch parquet splits. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Serialize, Deserialize)] +pub enum ParquetSplitKind { + Metrics, + Sketches, +} + +impl ParquetSplitKind { + /// Returns the prefix used when generating split IDs. + pub fn split_id_prefix(&self) -> &'static str { + match self { + Self::Metrics => "metrics_", + Self::Sketches => "sketches_", + } + } + + /// Returns the Postgres table name for this split kind. + pub fn table_name(&self) -> &'static str { + match self { + Self::Metrics => "metrics_splits", + Self::Sketches => "sketch_splits", + } + } + + /// Returns a human-readable label for logging and metrics. + pub fn label(&self) -> &'static str { + match self { + Self::Metrics => "metrics", + Self::Sketches => "sketch", + } + } +} + +impl std::fmt::Display for ParquetSplitKind { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.write_str(self.label()) + } +} + +/// Default kind used by serde for backwards-compatible deserialization. +fn default_metrics_kind() -> ParquetSplitKind { + ParquetSplitKind::Metrics +} + +/// Unique identifier for a parquet split (metrics or sketch). #[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] -pub struct SplitId(String); +pub struct ParquetSplitId(String); -impl SplitId { - /// Create a new SplitId from a string. +impl ParquetSplitId { + /// Create a new ParquetSplitId from a string. pub fn new(id: impl Into) -> Self { Self(id.into()) } - /// Generates a new unique SplitId using a ULID (timestamp + randomness). - pub fn generate() -> Self { + /// Generates a new unique split ID using a ULID (timestamp + randomness), + /// prefixed according to the split kind. + pub fn generate(kind: ParquetSplitKind) -> Self { Self(format!( - "metrics_{}", + "{}{}", + kind.split_id_prefix(), ulid::Ulid::new().to_string().to_lowercase() )) } @@ -50,13 +96,13 @@ impl SplitId { } } -impl std::fmt::Display for SplitId { +impl std::fmt::Display for ParquetSplitId { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { write!(f, "{}", self.0) } } -/// Time range covered by a metrics split. +/// Time range covered by a split. #[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] pub struct TimeRange { /// Start timestamp in seconds (inclusive). @@ -91,39 +137,15 @@ impl TimeRange { } } -/// State of a metrics split in the metastore. -#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] -pub enum MetricsSplitState { - /// Split is staged (being written, not yet queryable). - Staged, - /// Split is published (queryable). - Published, - /// Split is marked for deletion. - MarkedForDeletion, -} - -impl MetricsSplitState { - /// Returns a string representation for database storage. - pub fn as_str(&self) -> &'static str { - match self { - MetricsSplitState::Staged => "Staged", - MetricsSplitState::Published => "Published", - MetricsSplitState::MarkedForDeletion => "MarkedForDeletion", - } - } -} - -impl std::fmt::Display for MetricsSplitState { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - write!(f, "{}", self.as_str()) - } -} - -/// Metadata for a metrics split. +/// Unified metadata for a parquet split (metrics or sketch). #[derive(Debug, Clone, Serialize, Deserialize)] -pub struct MetricsSplitMetadata { +pub struct ParquetSplitMetadata { + /// What kind of split this is (metrics or sketches). + #[serde(default = "default_metrics_kind")] + pub kind: ParquetSplitKind, + /// Unique split identifier. - pub split_id: SplitId, + pub split_id: ParquetSplitId, /// Index unique identifier for Postgres foreign key relationship. pub index_uid: String, @@ -155,21 +177,31 @@ pub struct MetricsSplitMetadata { pub created_at: SystemTime, } -impl MetricsSplitMetadata { +impl ParquetSplitMetadata { /// Returns the parquet filename for this split, relative to the storage root. /// Always `{split_id}.parquet`. pub fn parquet_filename(&self) -> String { format!("{}.parquet", self.split_id) } + /// Returns the split ID as a string. + pub fn split_id_str(&self) -> &str { + self.split_id.as_str() + } + + /// Returns the size in bytes. + pub fn size_bytes(&self) -> u64 { + self.size_bytes + } + /// Cardinality threshold for routing tags to Postgres vs Parquet. /// Tags with < CARDINALITY_THRESHOLD unique values go to Postgres. /// Tags with >= CARDINALITY_THRESHOLD unique values use Parquet bloom filters. pub const CARDINALITY_THRESHOLD: usize = 1000; - /// Create a new MetricsSplitMetadata builder. - pub fn builder() -> MetricsSplitMetadataBuilder { - MetricsSplitMetadataBuilder::default() + /// Create a new ParquetSplitMetadata builder. + pub fn builder() -> ParquetSplitMetadataBuilder { + ParquetSplitMetadataBuilder::default() } /// Check if a tag key exceeds the cardinality threshold. @@ -210,10 +242,11 @@ impl MetricsSplitMetadata { } } -/// Builder for MetricsSplitMetadata. +/// Builder for ParquetSplitMetadata. #[derive(Default)] -pub struct MetricsSplitMetadataBuilder { - split_id: Option, +pub struct ParquetSplitMetadataBuilder { + kind: Option, + split_id: Option, index_uid: Option, time_range: Option, num_rows: u64, @@ -223,8 +256,13 @@ pub struct MetricsSplitMetadataBuilder { high_cardinality_tag_keys: HashSet, } -impl MetricsSplitMetadataBuilder { - pub fn split_id(mut self, id: SplitId) -> Self { +impl ParquetSplitMetadataBuilder { + pub fn kind(mut self, kind: ParquetSplitKind) -> Self { + self.kind = Some(kind); + self + } + + pub fn split_id(mut self, id: ParquetSplitId) -> Self { self.split_id = Some(id); self } @@ -284,9 +322,13 @@ impl MetricsSplitMetadataBuilder { self } - pub fn build(self) -> MetricsSplitMetadata { - MetricsSplitMetadata { - split_id: self.split_id.unwrap_or_else(SplitId::generate), + pub fn build(self) -> ParquetSplitMetadata { + let kind = self.kind.unwrap_or(ParquetSplitKind::Metrics); + ParquetSplitMetadata { + kind, + split_id: self + .split_id + .unwrap_or_else(|| ParquetSplitId::generate(kind)), index_uid: self.index_uid.expect("index_uid is required"), time_range: self.time_range.expect("time_range is required"), num_rows: self.num_rows, @@ -302,17 +344,32 @@ impl MetricsSplitMetadataBuilder { #[cfg(test)] mod tests { use super::*; + use crate::split::SplitState; #[test] fn test_split_id_generation() { - let id1 = SplitId::generate(); + let id1 = ParquetSplitId::generate(ParquetSplitKind::Metrics); // Sleep 1ms to ensure different timestamp std::thread::sleep(std::time::Duration::from_millis(1)); - let id2 = SplitId::generate(); + let id2 = ParquetSplitId::generate(ParquetSplitKind::Metrics); assert_ne!(id1.as_str(), id2.as_str()); assert!(id1.as_str().starts_with("metrics_")); } + #[test] + fn test_sketch_split_id_generation() { + let id = ParquetSplitId::generate(ParquetSplitKind::Sketches); + assert!(id.as_str().starts_with("sketches_")); + } + + #[test] + fn test_sketch_split_id_uniqueness() { + let id1 = ParquetSplitId::generate(ParquetSplitKind::Sketches); + std::thread::sleep(std::time::Duration::from_millis(1)); + let id2 = ParquetSplitId::generate(ParquetSplitKind::Sketches); + assert_ne!(id1.as_str(), id2.as_str()); + } + #[test] fn test_time_range_overlap() { let range1 = TimeRange::new(100, 200); @@ -327,7 +384,7 @@ mod tests { #[test] fn test_metadata_builder_with_tags() { - let metadata = MetricsSplitMetadata::builder() + let metadata = ParquetSplitMetadata::builder() .index_uid("test-index:00000000000000000000000000") .time_range(TimeRange::new(1000, 2000)) .add_metric_name("cpu.usage") @@ -337,6 +394,7 @@ mod tests { .add_high_cardinality_tag_key(TAG_HOST) .build(); + assert_eq!(metadata.kind, ParquetSplitKind::Metrics); assert_eq!(metadata.index_uid, "test-index:00000000000000000000000000"); assert!(metadata.metric_names.contains("cpu.usage")); assert_eq!(metadata.get_tag_values(TAG_SERVICE).unwrap().len(), 2); @@ -351,9 +409,61 @@ mod tests { assert!(!metadata.is_high_cardinality(TAG_SERVICE)); } + #[test] + fn test_sketch_metadata_builder() { + let metadata = ParquetSplitMetadata::builder() + .kind(ParquetSplitKind::Sketches) + .index_uid("sketch-index:00000000000000000000000000") + .time_range(TimeRange::new(1000, 2000)) + .add_metric_name("req.latency") + .add_low_cardinality_tag(TAG_SERVICE, "api") + .build(); + + assert_eq!(metadata.kind, ParquetSplitKind::Sketches); + assert!(metadata.split_id.as_str().starts_with("sketches_")); + assert!(metadata.metric_names.contains("req.latency")); + assert_eq!(metadata.time_range.start_secs, 1000); + } + + #[test] + fn test_sketch_metadata_serialization_roundtrip() { + let metadata = ParquetSplitMetadata::builder() + .kind(ParquetSplitKind::Sketches) + .index_uid("test:00000000000000000000000000") + .time_range(TimeRange::new(100, 200)) + .num_rows(500) + .size_bytes(1024) + .add_metric_name("latency") + .build(); + + let json = serde_json::to_string(&metadata).unwrap(); + let recovered: ParquetSplitMetadata = serde_json::from_str(&json).unwrap(); + assert_eq!(recovered.split_id.as_str(), metadata.split_id.as_str()); + assert_eq!(recovered.num_rows, 500); + assert_eq!(recovered.kind, ParquetSplitKind::Sketches); + } + + #[test] + fn test_backwards_compatible_deserialization() { + // Simulate JSON without the "kind" field (old format) + let json = r#"{ + "split_id": "metrics_test123", + "index_uid": "test:00000000000000000000000000", + "time_range": {"start_secs": 100, "end_secs": 200}, + "num_rows": 42, + "size_bytes": 1024, + "metric_names": [], + "low_cardinality_tags": {}, + "high_cardinality_tag_keys": [], + "created_at": {"secs_since_epoch": 0, "nanos_since_epoch": 0} + }"#; + let recovered: ParquetSplitMetadata = serde_json::from_str(json).unwrap(); + assert_eq!(recovered.kind, ParquetSplitKind::Metrics); + } + #[test] fn test_cardinality_promotion() { - let mut metadata = MetricsSplitMetadata::builder() + let mut metadata = ParquetSplitMetadata::builder() .index_uid("test-index:00000000000000000000000000") .time_range(TimeRange::new(1000, 2000)) .build(); @@ -378,7 +488,7 @@ mod tests { #[test] fn test_service_names_convenience() { - let metadata = MetricsSplitMetadata::builder() + let metadata = ParquetSplitMetadata::builder() .index_uid("test-index:00000000000000000000000000") .time_range(TimeRange::new(1000, 2000)) .add_low_cardinality_tag(TAG_SERVICE, "web") @@ -392,13 +502,20 @@ mod tests { } #[test] - fn test_metrics_split_state() { - assert_eq!(MetricsSplitState::Staged.as_str(), "Staged"); - assert_eq!(MetricsSplitState::Published.as_str(), "Published"); - assert_eq!( - MetricsSplitState::MarkedForDeletion.as_str(), - "MarkedForDeletion" - ); - assert_eq!(format!("{}", MetricsSplitState::Published), "Published"); + fn test_split_state() { + assert_eq!(SplitState::Staged.as_str(), "Staged"); + assert_eq!(SplitState::Published.as_str(), "Published"); + assert_eq!(SplitState::MarkedForDeletion.as_str(), "MarkedForDeletion"); + assert_eq!(format!("{}", SplitState::Published), "Published"); + } + + #[test] + fn test_split_kind_properties() { + assert_eq!(ParquetSplitKind::Metrics.split_id_prefix(), "metrics_"); + assert_eq!(ParquetSplitKind::Sketches.split_id_prefix(), "sketches_"); + assert_eq!(ParquetSplitKind::Metrics.table_name(), "metrics_splits"); + assert_eq!(ParquetSplitKind::Sketches.table_name(), "sketch_splits"); + assert_eq!(ParquetSplitKind::Metrics.label(), "metrics"); + assert_eq!(ParquetSplitKind::Sketches.label(), "sketch"); } } diff --git a/quickwit/quickwit-parquet-engine/src/split/mod.rs b/quickwit/quickwit-parquet-engine/src/split/mod.rs index 51a49dfff1c..3bdc5c77f02 100644 --- a/quickwit/quickwit-parquet-engine/src/split/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/split/mod.rs @@ -20,10 +20,14 @@ mod format; mod metadata; pub mod postgres; +pub mod state; pub use format::{CURRENT_FORMAT_VERSION, ParquetSplit}; pub use metadata::{ - MetricsSplitMetadata, MetricsSplitMetadataBuilder, MetricsSplitState, SplitId, TAG_DATACENTER, - TAG_ENV, TAG_HOST, TAG_REGION, TAG_SERVICE, TimeRange, + ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, ParquetSplitMetadataBuilder, + TAG_DATACENTER, TAG_ENV, TAG_HOST, TAG_REGION, TAG_SERVICE, TimeRange, }; -pub use postgres::{InsertableMetricsSplit, MetricsSplitRecord, MetricsSplits, PgMetricsSplit}; +pub use postgres::{ + InsertableParquetSplit, MetricsSplits, ParquetSplitRecord, PgParquetSplit, SketchSplits, +}; +pub use state::SplitState; diff --git a/quickwit/quickwit-parquet-engine/src/split/postgres.rs b/quickwit/quickwit-parquet-engine/src/split/postgres.rs index 8b2c8767846..06cfcd50067 100644 --- a/quickwit/quickwit-parquet-engine/src/split/postgres.rs +++ b/quickwit/quickwit-parquet-engine/src/split/postgres.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! PostgreSQL model for metrics_splits table. +//! PostgreSQL model for parquet split tables (metrics and sketches). //! //! This module provides the database model and conversion logic for storing -//! MetricsSplitMetadata in the Postgres metastore for Tier 1 pruning. +//! ParquetSplitMetadata in the Postgres metastore for Tier 1 pruning. use std::collections::{HashMap, HashSet}; @@ -23,9 +23,9 @@ use std::collections::{HashMap, HashSet}; use sea_query::Iden; use super::metadata::{ - MetricsSplitMetadata, MetricsSplitState, TAG_DATACENTER, TAG_ENV, TAG_HOST, TAG_REGION, - TAG_SERVICE, + ParquetSplitMetadata, TAG_DATACENTER, TAG_ENV, TAG_HOST, TAG_REGION, TAG_SERVICE, }; +use super::state::SplitState; /// Sea-query table identifier for metrics_splits. #[cfg_attr(feature = "postgres", derive(Iden))] @@ -34,7 +34,7 @@ pub enum MetricsSplits { Table, SplitId, SplitState, - IndexId, + IndexUid, TimeRangeStart, TimeRangeEnd, MetricNames, @@ -51,10 +51,34 @@ pub enum MetricsSplits { UpdateTimestamp, } -/// PostgreSQL row model for metrics_splits table. +/// Sea-query table identifier for sketch_splits. +#[cfg_attr(feature = "postgres", derive(Iden))] +#[derive(Clone, Copy)] +pub enum SketchSplits { + Table, + SplitId, + SplitState, + IndexUid, + TimeRangeStart, + TimeRangeEnd, + MetricNames, + TagService, + TagEnv, + TagDatacenter, + TagRegion, + TagHost, + HighCardinalityTagKeys, + NumRows, + SizeBytes, + SplitMetadataJson, + CreateTimestamp, + UpdateTimestamp, +} + +/// PostgreSQL row model for parquet split tables. /// Used for reading rows from the database. #[derive(Debug, Clone)] -pub struct PgMetricsSplit { +pub struct PgParquetSplit { pub split_id: String, pub split_state: String, pub index_uid: String, @@ -73,10 +97,10 @@ pub struct PgMetricsSplit { pub update_timestamp: i64, } -/// Insertable row for metrics_splits table. +/// Insertable row for parquet split tables. /// Used for writing rows to the database. #[derive(Debug, Clone)] -pub struct InsertableMetricsSplit { +pub struct InsertableParquetSplit { pub split_id: String, pub split_state: String, pub index_uid: String, @@ -94,11 +118,11 @@ pub struct InsertableMetricsSplit { pub split_metadata_json: String, } -impl InsertableMetricsSplit { - /// Convert MetricsSplitMetadata to an insertable row. +impl InsertableParquetSplit { + /// Convert ParquetSplitMetadata to an insertable row. pub fn from_metadata( - metadata: &MetricsSplitMetadata, - state: MetricsSplitState, + metadata: &ParquetSplitMetadata, + state: SplitState, ) -> Result { let split_metadata_json = serde_json::to_string(metadata)?; @@ -127,12 +151,12 @@ fn extract_tag_values(tags: &HashMap>, key: &str) -> Opt tags.get(key).map(|values| values.iter().cloned().collect()) } -impl PgMetricsSplit { - /// Convert database row to MetricsSplitMetadata. +impl PgParquetSplit { + /// Convert database row to ParquetSplitMetadata. /// Falls back to deserializing from JSON if row data is incomplete. - pub fn to_metadata(&self) -> Result { + pub fn to_metadata(&self) -> Result { // Primary path: deserialize from JSON (authoritative) - let metadata: MetricsSplitMetadata = serde_json::from_str(&self.split_metadata_json)?; + let metadata: ParquetSplitMetadata = serde_json::from_str(&self.split_metadata_json)?; // Overlay database columns (for consistency verification in debug builds) debug_assert_eq!(metadata.split_id.as_str(), self.split_id); @@ -143,31 +167,31 @@ impl PgMetricsSplit { } /// Parse the split state from the database string. - pub fn split_state(&self) -> Option { + pub fn split_state(&self) -> Option { match self.split_state.as_str() { - "Staged" => Some(MetricsSplitState::Staged), - "Published" => Some(MetricsSplitState::Published), - "MarkedForDeletion" => Some(MetricsSplitState::MarkedForDeletion), + "Staged" => Some(SplitState::Staged), + "Published" => Some(SplitState::Published), + "MarkedForDeletion" => Some(SplitState::MarkedForDeletion), _ => None, } } } -/// A complete metrics split record from the database. +/// A complete parquet split record from the database. #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] -pub struct MetricsSplitRecord { +pub struct ParquetSplitRecord { /// The split's state. - pub state: MetricsSplitState, + pub state: SplitState, /// Timestamp when the split was last updated. pub update_timestamp: i64, /// The split's immutable metadata. - pub metadata: MetricsSplitMetadata, + pub metadata: ParquetSplitMetadata, } -impl TryFrom for MetricsSplitRecord { +impl TryFrom for ParquetSplitRecord { type Error = String; - fn try_from(row: PgMetricsSplit) -> Result { + fn try_from(row: PgParquetSplit) -> Result { let state = row .split_state() .ok_or_else(|| format!("unknown split state: {}", row.split_state))?; @@ -186,12 +210,12 @@ impl TryFrom for MetricsSplitRecord { #[cfg(test)] mod tests { use super::*; - use crate::split::metadata::{MetricsSplitMetadata, SplitId, TimeRange}; + use crate::split::metadata::{ParquetSplitId, ParquetSplitKind, TimeRange}; #[test] fn test_insertable_from_metadata() { - let metadata = MetricsSplitMetadata::builder() - .split_id(SplitId::new("test-split-001")) + let metadata = ParquetSplitMetadata::builder() + .split_id(ParquetSplitId::new("test-split-001")) .index_uid("otel-metrics-v0_1:00000000000000000000000000") .time_range(TimeRange::new(1700000000, 1700003600)) .num_rows(50000) @@ -204,9 +228,8 @@ mod tests { .add_high_cardinality_tag_key(TAG_HOST) .build(); - let insertable = - InsertableMetricsSplit::from_metadata(&metadata, MetricsSplitState::Staged) - .expect("conversion should succeed"); + let insertable = InsertableParquetSplit::from_metadata(&metadata, SplitState::Staged) + .expect("conversion should succeed"); assert_eq!(insertable.split_id, "test-split-001"); assert_eq!(insertable.split_state, "Staged"); @@ -228,8 +251,8 @@ mod tests { #[test] fn test_pg_split_to_metadata_roundtrip() { - let original = MetricsSplitMetadata::builder() - .split_id(SplitId::new("roundtrip-test")) + let original = ParquetSplitMetadata::builder() + .split_id(ParquetSplitId::new("roundtrip-test")) .index_uid("test-index:00000000000000000000000000") .time_range(TimeRange::new(1000, 2000)) .num_rows(100) @@ -238,11 +261,10 @@ mod tests { .add_low_cardinality_tag(TAG_SERVICE, "test-service") .build(); - let insertable = - InsertableMetricsSplit::from_metadata(&original, MetricsSplitState::Published) - .expect("conversion should succeed"); + let insertable = InsertableParquetSplit::from_metadata(&original, SplitState::Published) + .expect("conversion should succeed"); - let pg_row = PgMetricsSplit { + let pg_row = PgParquetSplit { split_id: insertable.split_id, split_state: insertable.split_state, index_uid: insertable.index_uid, @@ -267,4 +289,73 @@ mod tests { assert_eq!(recovered.time_range, original.time_range); assert_eq!(recovered.num_rows, original.num_rows); } + + #[test] + fn test_sketch_insertable_from_metadata() { + let metadata = ParquetSplitMetadata::builder() + .kind(ParquetSplitKind::Sketches) + .split_id(ParquetSplitId::new("test-sketch-001")) + .index_uid("sketch-index:00000000000000000000000000") + .time_range(TimeRange::new(1700000000, 1700003600)) + .num_rows(50000) + .size_bytes(1024 * 1024) + .add_metric_name("req.latency") + .add_low_cardinality_tag(TAG_SERVICE, "api") + .build(); + + let insertable = + InsertableParquetSplit::from_metadata(&metadata, SplitState::Staged).unwrap(); + + assert_eq!(insertable.split_id, "test-sketch-001"); + assert_eq!(insertable.split_state, "Staged"); + assert_eq!(insertable.time_range_start, 1700000000); + assert_eq!(insertable.time_range_end, 1700003600); + assert!(insertable.metric_names.contains(&"req.latency".to_string())); + assert_eq!(insertable.tag_service.as_ref().unwrap().len(), 1); + } + + #[test] + fn test_pg_sketch_split_roundtrip() { + let original = ParquetSplitMetadata::builder() + .kind(ParquetSplitKind::Sketches) + .split_id(ParquetSplitId::new("roundtrip-test")) + .index_uid("test-index:00000000000000000000000000") + .time_range(TimeRange::new(1000, 2000)) + .num_rows(100) + .size_bytes(500) + .add_metric_name("test.metric") + .add_low_cardinality_tag(TAG_SERVICE, "test-service") + .build(); + + let insertable = + InsertableParquetSplit::from_metadata(&original, SplitState::Published).unwrap(); + + let pg_row = PgParquetSplit { + split_id: insertable.split_id, + split_state: insertable.split_state, + index_uid: insertable.index_uid, + time_range_start: insertable.time_range_start, + time_range_end: insertable.time_range_end, + metric_names: insertable.metric_names, + tag_service: insertable.tag_service, + tag_env: insertable.tag_env, + tag_datacenter: insertable.tag_datacenter, + tag_region: insertable.tag_region, + tag_host: insertable.tag_host, + high_cardinality_tag_keys: insertable.high_cardinality_tag_keys, + num_rows: insertable.num_rows, + size_bytes: insertable.size_bytes, + split_metadata_json: insertable.split_metadata_json, + update_timestamp: 1704067200, + }; + + let recovered = pg_row.to_metadata().unwrap(); + assert_eq!(recovered.split_id.as_str(), original.split_id.as_str()); + assert_eq!(recovered.index_uid, original.index_uid); + assert_eq!(recovered.time_range, original.time_range); + assert_eq!(recovered.num_rows, original.num_rows); + + let record = ParquetSplitRecord::try_from(pg_row).unwrap(); + assert_eq!(record.state, SplitState::Published); + } } diff --git a/quickwit/quickwit-parquet-engine/src/split/state.rs b/quickwit/quickwit-parquet-engine/src/split/state.rs new file mode 100644 index 00000000000..ef477a54630 --- /dev/null +++ b/quickwit/quickwit-parquet-engine/src/split/state.rs @@ -0,0 +1,45 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Shared split state enum used by both metrics and sketch splits. + +use serde::{Deserialize, Serialize}; + +/// State of a split in the metastore. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] +pub enum SplitState { + /// Split is staged (being written, not yet queryable). + Staged, + /// Split is published (queryable). + Published, + /// Split is marked for deletion. + MarkedForDeletion, +} + +impl SplitState { + /// Returns a string representation for database storage. + pub fn as_str(&self) -> &'static str { + match self { + SplitState::Staged => "Staged", + SplitState::Published => "Published", + SplitState::MarkedForDeletion => "MarkedForDeletion", + } + } +} + +impl std::fmt::Display for SplitState { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.as_str()) + } +} diff --git a/quickwit/quickwit-parquet-engine/src/storage/config.rs b/quickwit/quickwit-parquet-engine/src/storage/config.rs index 13b8d395a44..2eb63d73510 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/config.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/config.rs @@ -202,8 +202,6 @@ impl ParquetWriterConfig { #[cfg(test)] mod tests { - use std::sync::Arc; - use arrow::datatypes::Field; use super::*; diff --git a/quickwit/quickwit-parquet-engine/src/storage/mod.rs b/quickwit/quickwit-parquet-engine/src/storage/mod.rs index 0eb7a96391e..f7baa6575c1 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/mod.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/mod.rs @@ -12,10 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Storage layer for metrics Parquet files. -//! -//! Provides configurable Parquet writer with compression options -//! optimized for metrics workloads. +//! Storage layer for Parquet files. mod config; mod split_writer; diff --git a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs index 466e2c9cdcb..6e609697b98 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/split_writer.rs @@ -25,25 +25,28 @@ use tracing::{debug, info, instrument}; use super::config::ParquetWriterConfig; use super::writer::{ParquetWriteError, ParquetWriter}; -use crate::split::{MetricsSplitMetadata, ParquetSplit, SplitId, TAG_SERVICE, TimeRange}; +use crate::split::{ + ParquetSplitId, ParquetSplitKind, ParquetSplitMetadata, TAG_SERVICE, TimeRange, +}; -/// Writer that produces complete ParquetSplit with metadata from RecordBatch data. +/// Writer that produces Parquet split files with metadata from RecordBatch data. pub struct ParquetSplitWriter { - /// The underlying Parquet writer. + kind: ParquetSplitKind, writer: ParquetWriter, - /// Base directory for split files. base_path: PathBuf, } impl ParquetSplitWriter { /// Create a new ParquetSplitWriter. - /// - /// # Arguments - /// * `config` - Parquet writer configuration - /// * `base_path` - Directory where split files will be written - pub fn new(config: ParquetWriterConfig, base_path: impl Into) -> Self { + pub fn new( + kind: ParquetSplitKind, + config: ParquetWriterConfig, + sort_order: &'static [&'static str], + base_path: impl Into, + ) -> Self { Self { - writer: ParquetWriter::new(config), + kind, + writer: ParquetWriter::new(config, sort_order), base_path: base_path.into(), } } @@ -53,32 +56,20 @@ impl ParquetSplitWriter { &self.base_path } - /// Write a RecordBatch to a Parquet file and return a ParquetSplit with metadata. - /// - /// # Arguments - /// * `batch` - The RecordBatch to write - /// * `index_uid` - The index unique identifier for the split metadata - /// - /// # Returns - /// A ParquetSplit containing metadata extracted from the batch and the file path. - #[instrument(skip(self, batch), fields(batch_rows = batch.num_rows()))] + /// Write a RecordBatch to a Parquet file and return split metadata. + #[instrument(skip(self, batch), fields(batch_rows = batch.num_rows(), kind = %self.kind))] pub fn write_split( &self, batch: &RecordBatch, index_uid: &str, - ) -> Result { - // Generate unique split ID - let split_id = SplitId::generate(); - + ) -> Result { + let split_id = ParquetSplitId::generate(self.kind); let file_path = self.base_path.join(format!("{}.parquet", split_id)); - // Ensure the base directory exists std::fs::create_dir_all(&self.base_path)?; - // Write batch to file let size_bytes = self.writer.write_to_file(batch, &file_path)?; - // Extract time range from batch let time_range = extract_time_range(batch)?; debug!( start_secs = time_range.start_secs, @@ -86,29 +77,24 @@ impl ParquetSplitWriter { "extracted time range from batch" ); - // Extract distinct metric names from batch let metric_names = extract_metric_names(batch)?; + let service_names = extract_dict_column_values(batch, "service"); - // Extract distinct service names from batch - let service_names = extract_service_names(batch)?; - - // Build metadata - let metadata = MetricsSplitMetadata::builder() + let mut metadata = ParquetSplitMetadata::builder() + .kind(self.kind) .split_id(split_id.clone()) .index_uid(index_uid) .time_range(time_range) .num_rows(batch.num_rows() as u64) .size_bytes(size_bytes); - // Add metric names - let metadata = metric_names - .into_iter() - .fold(metadata, |m, name| m.add_metric_name(name)); + for name in metric_names { + metadata = metadata.add_metric_name(name); + } - // Add service names as low-cardinality tags - let metadata = service_names.into_iter().fold(metadata, |m, name| { - m.add_low_cardinality_tag(TAG_SERVICE, name) - }); + for name in service_names { + metadata = metadata.add_low_cardinality_tag(TAG_SERVICE, name); + } let metadata = metadata.build(); @@ -119,7 +105,7 @@ impl ParquetSplitWriter { "split file written successfully" ); - Ok(ParquetSplit::new(metadata)) + Ok(metadata) } } @@ -153,50 +139,29 @@ fn extract_metric_names(batch: &RecordBatch) -> Result, ParquetW .schema() .index_of("metric_name") .map_err(|_| ParquetWriteError::SchemaValidation("missing metric_name column".into()))?; - let metric_col = batch.column(metric_idx); - let mut names = HashSet::new(); + Ok(extract_dict_column_values_at(batch, metric_idx)) +} - // The column is Dictionary(Int32, Utf8) - if let Some(dict_array) = metric_col - .as_any() - .downcast_ref::>() - { - let values = dict_array.values(); - if let Some(string_values) = values.as_any().downcast_ref::() { - // Get all dictionary values that are actually used - for i in 0..dict_array.len() { - if !dict_array.is_null(i) - && let Ok(key) = dict_array.keys().value(i).try_into() - { - let key: usize = key; - if key < string_values.len() && !string_values.is_null(key) { - names.insert(string_values.value(key).to_string()); - } - } - } - } +/// Extracts distinct string values from a dictionary-encoded column by name. +/// Returns an empty set if the column doesn't exist or isn't dictionary-encoded. +fn extract_dict_column_values(batch: &RecordBatch, column_name: &str) -> HashSet { + match batch.schema().index_of(column_name).ok() { + Some(idx) => extract_dict_column_values_at(batch, idx), + None => HashSet::new(), } - - Ok(names) } -/// Extracts distinct service names from a RecordBatch. -fn extract_service_names(batch: &RecordBatch) -> Result, ParquetWriteError> { - let service_idx = match batch.schema().index_of("service").ok() { - Some(idx) => idx, - None => return Ok(HashSet::new()), - }; - let service_col = batch.column(service_idx); +/// Extracts distinct string values from a dictionary-encoded column by index. +fn extract_dict_column_values_at(batch: &RecordBatch, col_idx: usize) -> HashSet { + let col = batch.column(col_idx); let mut names = HashSet::new(); - // The column is Dictionary(Int32, Utf8) - if let Some(dict_array) = service_col + if let Some(dict_array) = col .as_any() .downcast_ref::>() { let values = dict_array.values(); if let Some(string_values) = values.as_any().downcast_ref::() { - // Get all dictionary values that are actually used for i in 0..dict_array.len() { if !dict_array.is_null(i) && let Ok(key) = dict_array.keys().value(i).try_into() @@ -210,7 +175,7 @@ fn extract_service_names(batch: &RecordBatch) -> Result, Parquet } } - Ok(names) + names } #[cfg(test)] @@ -221,6 +186,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; use super::*; + use crate::schema::SORT_ORDER; use crate::test_helpers::{create_dict_array, create_nullable_dict_array}; /// Create a test batch with required fields, optional service column, and specified tag @@ -296,21 +262,26 @@ mod tests { let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(config, temp_dir.path()); + let writer = ParquetSplitWriter::new( + ParquetSplitKind::Metrics, + config, + SORT_ORDER, + temp_dir.path(), + ); let batch = create_test_batch(10); - let split = writer.write_split(&batch, "test-index").unwrap(); + let metadata = writer.write_split(&batch, "test-index").unwrap(); // Verify file exists - let file_path = temp_dir.path().join(split.metadata.parquet_filename()); + let file_path = temp_dir.path().join(metadata.parquet_filename()); assert!( std::fs::metadata(&file_path).is_ok(), "Parquet file should exist" ); // Verify metadata - assert_eq!(split.metadata.num_rows, 10); - assert!(split.metadata.size_bytes > 0); + assert_eq!(metadata.num_rows, 10); + assert!(metadata.size_bytes > 0); } #[test] @@ -318,7 +289,12 @@ mod tests { let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(config, temp_dir.path()); + let writer = ParquetSplitWriter::new( + ParquetSplitKind::Metrics, + config, + SORT_ORDER, + temp_dir.path(), + ); // Create batch with timestamps [100, 150, 200] let batch = create_test_batch_with_options( @@ -328,11 +304,10 @@ mod tests { Some(&["my-service", "my-service", "my-service"]), &[], ); - let split = writer.write_split(&batch, "test-index").unwrap(); + let metadata = writer.write_split(&batch, "test-index").unwrap(); - // Verify time range - assert_eq!(split.metadata.time_range.start_secs, 100); - assert_eq!(split.metadata.time_range.end_secs, 201); // exclusive + assert_eq!(metadata.time_range.start_secs, 100); + assert_eq!(metadata.time_range.end_secs, 201); // exclusive } #[test] @@ -340,7 +315,12 @@ mod tests { let config = ParquetWriterConfig::default(); let temp_dir = tempfile::tempdir().unwrap(); - let writer = ParquetSplitWriter::new(config, temp_dir.path()); + let writer = ParquetSplitWriter::new( + ParquetSplitKind::Metrics, + config, + SORT_ORDER, + temp_dir.path(), + ); // Create batch with specific metric names let batch = create_test_batch_with_options( @@ -350,11 +330,10 @@ mod tests { Some(&["my-service", "my-service", "my-service"]), &[], ); - let split = writer.write_split(&batch, "test-index").unwrap(); + let metadata = writer.write_split(&batch, "test-index").unwrap(); - // Verify metric names (distinct values) - assert!(split.metadata.metric_names.contains("cpu.usage")); - assert!(split.metadata.metric_names.contains("memory.used")); - assert_eq!(split.metadata.metric_names.len(), 2); + assert!(metadata.metric_names.contains("cpu.usage")); + assert!(metadata.metric_names.contains("memory.used")); + assert_eq!(metadata.metric_names.len(), 2); } } diff --git a/quickwit/quickwit-parquet-engine/src/storage/writer.rs b/quickwit/quickwit-parquet-engine/src/storage/writer.rs index 6f29c0be4cc..abbdd79cfca 100644 --- a/quickwit/quickwit-parquet-engine/src/storage/writer.rs +++ b/quickwit/quickwit-parquet-engine/src/storage/writer.rs @@ -27,7 +27,6 @@ use thiserror::Error; use tracing::{debug, instrument}; use super::config::ParquetWriterConfig; -use crate::schema::{SORT_ORDER, validate_required_fields}; /// Errors that can occur during parquet writing. #[derive(Debug, Error)] @@ -49,15 +48,16 @@ pub enum ParquetWriteError { SchemaValidation(String), } -/// Writer for metrics data to Parquet format. +/// Writer for data to Parquet format with configurable sort order. pub struct ParquetWriter { config: ParquetWriterConfig, + sort_order: &'static [&'static str], } impl ParquetWriter { - /// Create a new ParquetWriter. - pub fn new(config: ParquetWriterConfig) -> Self { - Self { config } + /// Create a new ParquetWriter with a custom sort order. + pub fn new(config: ParquetWriterConfig, sort_order: &'static [&'static str]) -> Self { + Self { config, sort_order } } /// Get the writer configuration. @@ -70,7 +70,8 @@ impl ParquetWriter { /// missing columns are skipped. fn sort_batch(&self, batch: &RecordBatch) -> Result { let schema = batch.schema(); - let mut sort_columns: Vec = SORT_ORDER + let mut sort_columns: Vec = self + .sort_order .iter() .filter_map(|name| schema.index_of(name).ok()) .map(|idx| SortColumn { @@ -107,12 +108,9 @@ impl ParquetWriter { } /// Write a RecordBatch to Parquet bytes in memory. - /// The batch is sorted before writing by: metric_name, common tags, timestamp. + /// The batch is sorted before writing by the configured sort order. #[instrument(skip(self, batch), fields(batch_rows = batch.num_rows()))] pub fn write_to_bytes(&self, batch: &RecordBatch) -> Result, ParquetWriteError> { - validate_required_fields(&batch.schema()) - .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string()))?; - // Sort the batch before writing for efficient pruning let sorted_batch = self.sort_batch(batch)?; @@ -129,7 +127,7 @@ impl ParquetWriter { } /// Write a RecordBatch to a Parquet file. - /// The batch is sorted before writing by: metric_name, common tags, timestamp. + /// The batch is sorted before writing by the configured sort order. /// /// Returns the number of bytes written. #[instrument(skip(self, batch), fields(batch_rows = batch.num_rows(), path = %path.display()))] @@ -138,9 +136,6 @@ impl ParquetWriter { batch: &RecordBatch, path: &Path, ) -> Result { - validate_required_fields(&batch.schema()) - .map_err(|e| ParquetWriteError::SchemaValidation(e.to_string()))?; - // Sort the batch before writing for efficient pruning let sorted_batch = self.sort_batch(batch)?; @@ -167,6 +162,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Int32Type, Schema}; use super::*; + use crate::schema::SORT_ORDER; use crate::test_helpers::create_test_batch_with_tags; fn create_test_batch() -> RecordBatch { @@ -176,13 +172,13 @@ mod tests { #[test] fn test_writer_creation() { let config = ParquetWriterConfig::default(); - let _writer = ParquetWriter::new(config); + let _writer = ParquetWriter::new(config, SORT_ORDER); } #[test] fn test_write_to_bytes() { let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); let batch = create_test_batch(); let bytes = writer.write_to_bytes(&batch).unwrap(); @@ -195,7 +191,7 @@ mod tests { #[test] fn test_write_to_file() { let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); let batch = create_test_batch(); let temp_dir = std::env::temp_dir(); @@ -211,7 +207,7 @@ mod tests { #[test] fn test_schema_validation_missing_field() { let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); // Create a batch missing required fields let wrong_schema = Arc::new(Schema::new(vec![Field::new( @@ -235,7 +231,7 @@ mod tests { #[test] fn test_schema_validation_wrong_type() { let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); // Create a batch where metric_name has wrong type (Utf8 instead of Dictionary) let wrong_schema = Arc::new(Schema::new(vec![ @@ -267,7 +263,7 @@ mod tests { use super::super::config::Compression; let config = ParquetWriterConfig::new().with_compression(Compression::Snappy); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); let batch = create_test_batch(); let bytes = writer.write_to_bytes(&batch).unwrap(); @@ -283,7 +279,7 @@ mod tests { use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; let config = ParquetWriterConfig::default(); - let writer = ParquetWriter::new(config); + let writer = ParquetWriter::new(config, SORT_ORDER); // Create a schema with required fields + service tag for sort verification let schema = Arc::new(Schema::new(vec![ diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 206d835537c..21b1439dcea 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -224,6 +224,21 @@ service MetastoreService { // Deletes metrics splits. rpc DeleteMetricsSplits(DeleteMetricsSplitsRequest) returns (EmptyResponse); + + // Stages sketch splits (DDSketch). + rpc StageSketchSplits(StageSketchSplitsRequest) returns (EmptyResponse); + + // Publishes sketch splits. + rpc PublishSketchSplits(PublishSketchSplitsRequest) returns (EmptyResponse); + + // Lists sketch splits. + rpc ListSketchSplits(ListSketchSplitsRequest) returns (ListSketchSplitsResponse); + + // Marks sketch splits for deletion. + rpc MarkSketchSplitsForDeletion(MarkSketchSplitsForDeletionRequest) returns (EmptyResponse); + + // Deletes sketch splits. + rpc DeleteSketchSplits(DeleteSketchSplitsRequest) returns (EmptyResponse); } message EmptyResponse { @@ -634,3 +649,37 @@ message DeleteMetricsSplitsRequest { quickwit.common.IndexUid index_uid = 1; repeated string split_ids = 2; } + +// --- DDSketch split messages --- + +message StageSketchSplitsRequest { + quickwit.common.IndexUid index_uid = 1; + repeated string splits_metadata_json = 2; +} + +message PublishSketchSplitsRequest { + quickwit.common.IndexUid index_uid = 1; + repeated string staged_split_ids = 2; + repeated string replaced_split_ids = 3; + optional string index_checkpoint_delta_json_opt = 4; + optional string publish_token_opt = 5; +} + +message ListSketchSplitsRequest { + quickwit.common.IndexUid index_uid = 1; + string query_json = 2; +} + +message ListSketchSplitsResponse { + repeated string splits_serialized_json = 1; +} + +message MarkSketchSplitsForDeletionRequest { + quickwit.common.IndexUid index_uid = 1; + repeated string split_ids = 2; +} + +message DeleteSketchSplitsRequest { + quickwit.common.IndexUid index_uid = 1; + repeated string split_ids = 2; +} diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 2ce790ebbf9..e05001fa7e9 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -596,6 +596,60 @@ pub struct DeleteMetricsSplitsRequest { 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 StageSketchSplitsRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, repeated, tag = "2")] + pub splits_metadata_json: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct PublishSketchSplitsRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, repeated, tag = "2")] + pub staged_split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(string, repeated, tag = "3")] + pub replaced_split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + #[prost(string, optional, tag = "4")] + pub index_checkpoint_delta_json_opt: ::core::option::Option< + ::prost::alloc::string::String, + >, + #[prost(string, optional, tag = "5")] + pub publish_token_opt: ::core::option::Option<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct ListSketchSplitsRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "2")] + pub query_json: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct ListSketchSplitsResponse { + #[prost(string, repeated, tag = "1")] + pub splits_serialized_json: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct MarkSketchSplitsForDeletionRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, repeated, tag = "2")] + 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 DeleteSketchSplitsRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, repeated, tag = "2")] + pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, +} +#[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)] @@ -886,6 +940,31 @@ impl RpcName for DeleteMetricsSplitsRequest { "delete_metrics_splits" } } +impl RpcName for StageSketchSplitsRequest { + fn rpc_name() -> &'static str { + "stage_sketch_splits" + } +} +impl RpcName for PublishSketchSplitsRequest { + fn rpc_name() -> &'static str { + "publish_sketch_splits" + } +} +impl RpcName for ListSketchSplitsRequest { + fn rpc_name() -> &'static str { + "list_sketch_splits" + } +} +impl RpcName for MarkSketchSplitsForDeletionRequest { + fn rpc_name() -> &'static str { + "mark_sketch_splits_for_deletion" + } +} +impl RpcName for DeleteSketchSplitsRequest { + fn rpc_name() -> &'static str { + "delete_sketch_splits" + } +} pub type MetastoreServiceStream = quickwit_common::ServiceStream< crate::metastore::MetastoreResult, >; @@ -1098,6 +1177,31 @@ pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { &self, request: DeleteMetricsSplitsRequest, ) -> crate::metastore::MetastoreResult; + ///Stages sketch splits (DDSketch). + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult; + ///Publishes sketch splits. + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult; + ///Lists sketch splits. + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult; + ///Marks sketch splits for deletion. + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult; + ///Deletes sketch splits. + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult; async fn check_connectivity(&self) -> anyhow::Result<()>; fn endpoints(&self) -> Vec; } @@ -1436,6 +1540,36 @@ impl MetastoreService for MetastoreServiceClient { ) -> crate::metastore::MetastoreResult { self.inner.0.delete_metrics_splits(request).await } + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.stage_sketch_splits(request).await + } + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.publish_sketch_splits(request).await + } + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.list_sketch_splits(request).await + } + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.mark_sketch_splits_for_deletion(request).await + } + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.delete_sketch_splits(request).await + } async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.0.check_connectivity().await } @@ -1684,6 +1818,36 @@ pub mod mock_metastore_service { ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_metrics_splits(request).await } + async fn stage_sketch_splits( + &self, + request: super::StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.stage_sketch_splits(request).await + } + async fn publish_sketch_splits( + &self, + request: super::PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.publish_sketch_splits(request).await + } + async fn list_sketch_splits( + &self, + request: super::ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_sketch_splits(request).await + } + async fn mark_sketch_splits_for_deletion( + &self, + request: super::MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.mark_sketch_splits_for_deletion(request).await + } + async fn delete_sketch_splits( + &self, + request: super::DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_sketch_splits(request).await + } async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.lock().await.check_connectivity().await } @@ -2304,6 +2468,86 @@ impl tower::Service for InnerMetastoreServiceClient Box::pin(fut) } } +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + 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: StageSketchSplitsRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.stage_sketch_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + 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: PublishSketchSplitsRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.publish_sketch_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = ListSketchSplitsResponse; + type Error = crate::metastore::MetastoreError; + 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: ListSketchSplitsRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.list_sketch_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + 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: MarkSketchSplitsForDeletionRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.mark_sketch_splits_for_deletion(request).await }; + Box::pin(fut) + } +} +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + 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: DeleteSketchSplitsRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.delete_sketch_splits(request).await }; + Box::pin(fut) + } +} /// A tower service stack is a set of tower services. #[derive(Debug)] struct MetastoreServiceTowerServiceStack { @@ -2499,6 +2743,31 @@ struct MetastoreServiceTowerServiceStack { EmptyResponse, crate::metastore::MetastoreError, >, + stage_sketch_splits_svc: quickwit_common::tower::BoxService< + StageSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + publish_sketch_splits_svc: quickwit_common::tower::BoxService< + PublishSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + list_sketch_splits_svc: quickwit_common::tower::BoxService< + ListSketchSplitsRequest, + ListSketchSplitsResponse, + crate::metastore::MetastoreError, + >, + mark_sketch_splits_for_deletion_svc: quickwit_common::tower::BoxService< + MarkSketchSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + delete_sketch_splits_svc: quickwit_common::tower::BoxService< + DeleteSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, } #[async_trait::async_trait] impl MetastoreService for MetastoreServiceTowerServiceStack { @@ -2735,6 +3004,41 @@ impl MetastoreService for MetastoreServiceTowerServiceStack { ) -> crate::metastore::MetastoreResult { self.delete_metrics_splits_svc.clone().ready().await?.call(request).await } + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.stage_sketch_splits_svc.clone().ready().await?.call(request).await + } + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.publish_sketch_splits_svc.clone().ready().await?.call(request).await + } + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.list_sketch_splits_svc.clone().ready().await?.call(request).await + } + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.mark_sketch_splits_for_deletion_svc + .clone() + .ready() + .await? + .call(request) + .await + } + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_sketch_splits_svc.clone().ready().await?.call(request).await + } async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.0.check_connectivity().await } @@ -3122,6 +3426,56 @@ type DeleteMetricsSplitsLayer = quickwit_common::tower::BoxLayer< EmptyResponse, crate::metastore::MetastoreError, >; +type StageSketchSplitsLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + StageSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + StageSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; +type PublishSketchSplitsLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + PublishSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + PublishSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; +type ListSketchSplitsLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + ListSketchSplitsRequest, + ListSketchSplitsResponse, + crate::metastore::MetastoreError, + >, + ListSketchSplitsRequest, + ListSketchSplitsResponse, + crate::metastore::MetastoreError, +>; +type MarkSketchSplitsForDeletionLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + MarkSketchSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + MarkSketchSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; +type DeleteSketchSplitsLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + DeleteSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + DeleteSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; #[derive(Debug, Default)] pub struct MetastoreServiceTowerLayerStack { create_index_layers: Vec, @@ -3162,6 +3516,11 @@ pub struct MetastoreServiceTowerLayerStack { list_metrics_splits_layers: Vec, mark_metrics_splits_for_deletion_layers: Vec, delete_metrics_splits_layers: Vec, + stage_sketch_splits_layers: Vec, + publish_sketch_splits_layers: Vec, + list_sketch_splits_layers: Vec, + mark_sketch_splits_for_deletion_layers: Vec, + delete_sketch_splits_layers: Vec, } impl MetastoreServiceTowerLayerStack { pub fn stack_layer(mut self, layer: L) -> Self @@ -4142,32 +4501,163 @@ impl MetastoreServiceTowerLayerStack { >>::Service as tower::Service< DeleteMetricsSplitsRequest, >>::Future: Send + 'static, - { - self.create_index_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.update_index_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.index_metadata_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.indexes_metadata_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.list_indexes_metadata_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.delete_index_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.list_index_stats_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.list_splits_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.stage_splits_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.publish_splits_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.mark_splits_for_deletion_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.delete_splits_layers - .push(quickwit_common::tower::BoxLayer::new(layer.clone())); - self.add_source_layers + L: tower::Layer< + quickwit_common::tower::BoxService< + StageSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + StageSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + PublishSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + PublishSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + PublishSketchSplitsRequest, + >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + ListSketchSplitsRequest, + ListSketchSplitsResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + ListSketchSplitsRequest, + Response = ListSketchSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + MarkSketchSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + MarkSketchSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + MarkSketchSplitsForDeletionRequest, + >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + DeleteSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + DeleteSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + DeleteSketchSplitsRequest, + >>::Future: Send + 'static, + { + self.create_index_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.update_index_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.index_metadata_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.indexes_metadata_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.list_indexes_metadata_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.delete_index_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.list_index_stats_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.list_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.stage_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.publish_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.mark_splits_for_deletion_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.delete_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.add_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.update_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); @@ -4219,6 +4709,16 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.delete_metrics_splits_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.stage_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.publish_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.list_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.mark_sketch_splits_for_deletion_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.delete_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self } pub fn stack_create_index_layer(mut self, layer: L) -> Self @@ -4986,6 +5486,112 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_stage_sketch_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + StageSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + StageSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.stage_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_publish_sketch_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + PublishSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + PublishSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.publish_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_list_sketch_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + ListSketchSplitsRequest, + ListSketchSplitsResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + ListSketchSplitsRequest, + Response = ListSketchSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.list_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_mark_sketch_splits_for_deletion_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + MarkSketchSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + MarkSketchSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.mark_sketch_splits_for_deletion_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stack_delete_sketch_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + DeleteSketchSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + DeleteSketchSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_sketch_splits_layers + .push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn build(self, instance: T) -> MetastoreServiceClient where T: MetastoreService, @@ -5350,6 +5956,46 @@ impl MetastoreServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let stage_sketch_splits_svc = self + .stage_sketch_splits_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let publish_sketch_splits_svc = self + .publish_sketch_splits_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let list_sketch_splits_svc = self + .list_sketch_splits_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let mark_sketch_splits_for_deletion_svc = self + .mark_sketch_splits_for_deletion_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); + let delete_sketch_splits_svc = self + .delete_sketch_splits_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let tower_svc_stack = MetastoreServiceTowerServiceStack { inner: inner_client, create_index_svc, @@ -5390,6 +6036,11 @@ impl MetastoreServiceTowerLayerStack { list_metrics_splits_svc, mark_metrics_splits_for_deletion_svc, delete_metrics_splits_svc, + stage_sketch_splits_svc, + publish_sketch_splits_svc, + list_sketch_splits_svc, + mark_sketch_splits_for_deletion_svc, + delete_sketch_splits_svc, }; MetastoreServiceClient::new(tower_svc_stack) } @@ -5689,34 +6340,67 @@ where >, > + tower::Service< - StageMetricsSplitsRequest, + StageMetricsSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + PublishMetricsSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ListMetricsSplitsRequest, + Response = ListMetricsSplitsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture< + ListMetricsSplitsResponse, + crate::metastore::MetastoreError, + >, + > + + tower::Service< + MarkMetricsSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + DeleteMetricsSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + StageSketchSplitsRequest, Response = EmptyResponse, Error = crate::metastore::MetastoreError, Future = BoxFuture, > + tower::Service< - PublishMetricsSplitsRequest, + PublishSketchSplitsRequest, Response = EmptyResponse, Error = crate::metastore::MetastoreError, Future = BoxFuture, > + tower::Service< - ListMetricsSplitsRequest, - Response = ListMetricsSplitsResponse, + ListSketchSplitsRequest, + Response = ListSketchSplitsResponse, Error = crate::metastore::MetastoreError, Future = BoxFuture< - ListMetricsSplitsResponse, + ListSketchSplitsResponse, crate::metastore::MetastoreError, >, > + tower::Service< - MarkMetricsSplitsForDeletionRequest, + MarkSketchSplitsForDeletionRequest, Response = EmptyResponse, Error = crate::metastore::MetastoreError, Future = BoxFuture, > + tower::Service< - DeleteMetricsSplitsRequest, + DeleteSketchSplitsRequest, Response = EmptyResponse, Error = crate::metastore::MetastoreError, Future = BoxFuture, @@ -5950,6 +6634,36 @@ where ) -> crate::metastore::MetastoreResult { self.clone().call(request).await } + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } async fn check_connectivity(&self) -> anyhow::Result<()> { if self.inner.is_disconnected() { anyhow::bail!("actor `{}` is disconnected", self.inner.actor_instance_id()) @@ -6537,6 +7251,76 @@ where DeleteMetricsSplitsRequest::rpc_name(), )) } + async fn stage_sketch_splits( + &self, + request: StageSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .stage_sketch_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + StageSketchSplitsRequest::rpc_name(), + )) + } + async fn publish_sketch_splits( + &self, + request: PublishSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .publish_sketch_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + PublishSketchSplitsRequest::rpc_name(), + )) + } + async fn list_sketch_splits( + &self, + request: ListSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .list_sketch_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + ListSketchSplitsRequest::rpc_name(), + )) + } + async fn mark_sketch_splits_for_deletion( + &self, + request: MarkSketchSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .mark_sketch_splits_for_deletion(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + MarkSketchSplitsForDeletionRequest::rpc_name(), + )) + } + async fn delete_sketch_splits( + &self, + request: DeleteSketchSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .delete_sketch_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + DeleteSketchSplitsRequest::rpc_name(), + )) + } async fn check_connectivity(&self) -> anyhow::Result<()> { if self.connection_addrs_rx.borrow().is_empty() { anyhow::bail!("no server currently available") @@ -6993,6 +7777,61 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn stage_sketch_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .stage_sketch_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn publish_sketch_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .publish_sketch_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn list_sketch_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .list_sketch_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn mark_sketch_splits_for_deletion( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .mark_sketch_splits_for_deletion(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } + async fn delete_sketch_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .delete_sketch_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } } /// Generated client implementations. pub mod metastore_service_grpc_client { @@ -8064,22 +8903,160 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/GetClusterIdentity", + "/quickwit.metastore.MetastoreService/GetClusterIdentity", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "GetClusterIdentity", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Stages metrics splits in the metastore. + pub async fn stage_metrics_splits( + &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.metastore.MetastoreService/StageMetricsSplits", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "StageMetricsSplits", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Publishes metrics splits. + pub async fn publish_metrics_splits( + &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.metastore.MetastoreService/PublishMetricsSplits", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "PublishMetricsSplits", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Lists metrics splits. + pub async fn list_metrics_splits( + &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.metastore.MetastoreService/ListMetricsSplits", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "ListMetricsSplits", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Marks metrics splits for deletion. + pub async fn mark_metrics_splits_for_deletion( + &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.metastore.MetastoreService/MarkMetricsSplitsForDeletion", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "MarkMetricsSplitsForDeletion", + ), + ); + self.inner.unary(req, path, codec).await + } + /// Deletes metrics splits. + pub async fn delete_metrics_splits( + &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.metastore.MetastoreService/DeleteMetricsSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "GetClusterIdentity", + "DeleteMetricsSplits", ), ); self.inner.unary(req, path, codec).await } - /// Stages metrics splits in the metastore. - pub async fn stage_metrics_splits( + /// Stages sketch splits (DDSketch). + pub async fn stage_sketch_splits( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result, tonic::Status> { self.inner .ready() @@ -8091,22 +9068,22 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/StageMetricsSplits", + "/quickwit.metastore.MetastoreService/StageSketchSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "StageMetricsSplits", + "StageSketchSplits", ), ); self.inner.unary(req, path, codec).await } - /// Publishes metrics splits. - pub async fn publish_metrics_splits( + /// Publishes sketch splits. + pub async fn publish_sketch_splits( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result, tonic::Status> { self.inner .ready() @@ -8118,24 +9095,24 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/PublishMetricsSplits", + "/quickwit.metastore.MetastoreService/PublishSketchSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "PublishMetricsSplits", + "PublishSketchSplits", ), ); self.inner.unary(req, path, codec).await } - /// Lists metrics splits. - pub async fn list_metrics_splits( + /// Lists sketch splits. + pub async fn list_sketch_splits( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -8148,22 +9125,22 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/ListMetricsSplits", + "/quickwit.metastore.MetastoreService/ListSketchSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "ListMetricsSplits", + "ListSketchSplits", ), ); self.inner.unary(req, path, codec).await } - /// Marks metrics splits for deletion. - pub async fn mark_metrics_splits_for_deletion( + /// Marks sketch splits for deletion. + pub async fn mark_sketch_splits_for_deletion( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result, tonic::Status> { self.inner .ready() @@ -8175,22 +9152,22 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/MarkMetricsSplitsForDeletion", + "/quickwit.metastore.MetastoreService/MarkSketchSplitsForDeletion", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "MarkMetricsSplitsForDeletion", + "MarkSketchSplitsForDeletion", ), ); self.inner.unary(req, path, codec).await } - /// Deletes metrics splits. - pub async fn delete_metrics_splits( + /// Deletes sketch splits. + pub async fn delete_sketch_splits( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result, tonic::Status> { self.inner .ready() @@ -8202,14 +9179,14 @@ pub mod metastore_service_grpc_client { })?; let codec = tonic_prost::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/DeleteMetricsSplits", + "/quickwit.metastore.MetastoreService/DeleteSketchSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "DeleteMetricsSplits", + "DeleteSketchSplits", ), ); self.inner.unary(req, path, codec).await @@ -8498,6 +9475,34 @@ pub mod metastore_service_grpc_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; + /// Stages sketch splits (DDSketch). + async fn stage_sketch_splits( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Publishes sketch splits. + async fn publish_sketch_splits( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Lists sketch splits. + async fn list_sketch_splits( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// Marks sketch splits for deletion. + async fn mark_sketch_splits_for_deletion( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Deletes sketch splits. + async fn delete_sketch_splits( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; } /// Metastore meant to manage Quickwit's indexes, their splits and delete tasks. /// @@ -10456,6 +11461,256 @@ pub mod metastore_service_grpc_server { }; Box::pin(fut) } + "/quickwit.metastore.MetastoreService/StageSketchSplits" => { + #[allow(non_camel_case_types)] + struct StageSketchSplitsSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for StageSketchSplitsSvc { + type Response = super::EmptyResponse; + 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 { + ::stage_sketch_splits( + &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 = StageSketchSplitsSvc(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.metastore.MetastoreService/PublishSketchSplits" => { + #[allow(non_camel_case_types)] + struct PublishSketchSplitsSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for PublishSketchSplitsSvc { + type Response = super::EmptyResponse; + 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 { + ::publish_sketch_splits( + &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 = PublishSketchSplitsSvc(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.metastore.MetastoreService/ListSketchSplits" => { + #[allow(non_camel_case_types)] + struct ListSketchSplitsSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for ListSketchSplitsSvc { + type Response = super::ListSketchSplitsResponse; + 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 { + ::list_sketch_splits( + &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 = ListSketchSplitsSvc(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.metastore.MetastoreService/MarkSketchSplitsForDeletion" => { + #[allow(non_camel_case_types)] + struct MarkSketchSplitsForDeletionSvc( + pub Arc, + ); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService< + super::MarkSketchSplitsForDeletionRequest, + > for MarkSketchSplitsForDeletionSvc { + type Response = super::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request< + super::MarkSketchSplitsForDeletionRequest, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + ::mark_sketch_splits_for_deletion( + &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 = MarkSketchSplitsForDeletionSvc(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.metastore.MetastoreService/DeleteSketchSplits" => { + #[allow(non_camel_case_types)] + struct DeleteSketchSplitsSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for DeleteSketchSplitsSvc { + type Response = super::EmptyResponse; + 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 { + ::delete_sketch_splits( + &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 = DeleteSketchSplitsSvc(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( diff --git a/quickwit/quickwit-proto/src/getters.rs b/quickwit/quickwit-proto/src/getters.rs index 9cf5bd77a26..c1be955c412 100644 --- a/quickwit/quickwit-proto/src/getters.rs +++ b/quickwit/quickwit-proto/src/getters.rs @@ -144,7 +144,14 @@ generate_getters! { PublishMetricsSplitsRequest, ListMetricsSplitsRequest, MarkMetricsSplitsForDeletionRequest, - DeleteMetricsSplitsRequest + DeleteMetricsSplitsRequest, + + // Sketch Split API + StageSketchSplitsRequest, + PublishSketchSplitsRequest, + ListSketchSplitsRequest, + MarkSketchSplitsForDeletionRequest, + DeleteSketchSplitsRequest } // [`PipelineUid`] getters