From b43cb972a595a57920faebf44637570ebbe840c9 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Fri, 21 Jul 2023 09:11:05 +0900 Subject: [PATCH] WIP --- quickwit/quickwit-indexing/Cargo.toml | 1 + .../src/actors/doc_processor.rs | 27 +- .../src/actors/indexing_pipeline.rs | 18 +- .../src/actors/indexing_service.rs | 91 ++-- .../src/actors/merge_executor.rs | 51 +- .../src/actors/merge_pipeline.rs | 22 +- .../quickwit-indexing/src/actors/publisher.rs | 30 +- .../quickwit-indexing/src/actors/uploader.rs | 16 +- .../src/models/indexing_service_message.rs | 3 +- .../src/source/kafka_source.rs | 32 +- quickwit/quickwit-indexing/src/test_utils.rs | 11 +- .../src/actors/delete_task_pipeline.rs | 9 +- .../src/actors/delete_task_planner.rs | 90 ++-- .../src/actors/delete_task_service.rs | 23 +- .../src/actors/garbage_collector.rs | 75 +-- .../src/actors/retention_policy_executor.rs | 23 +- .../src/garbage_collection.rs | 97 ++-- .../src/retention_policy_execution.rs | 31 +- .../quickwit-metastore-utils/src/bin/proxy.rs | 16 +- quickwit/quickwit-metastore/src/checkpoint.rs | 45 +- quickwit/quickwit-metastore/src/lib.rs | 20 +- .../file_backed_index/mod.rs | 141 +++--- .../metastore/file_backed_metastore/mod.rs | 306 +++++++----- .../metastore/grpc_metastore/grpc_adapter.rs | 407 +++++----------- .../src/metastore/grpc_metastore/mod.rs | 340 +++---------- .../src/metastore/index_metadata/mod.rs | 30 +- .../src/metastore/index_metadata/serialize.rs | 9 +- .../src/metastore/instrumented_metastore.rs | 191 +++----- .../metastore/metastore_event_publisher.rs | 227 ++++----- .../quickwit-metastore/src/metastore/mod.rs | 453 +++++++++++++----- .../src/metastore/postgresql_metastore.rs | 421 ++++++++-------- .../src/metastore/postgresql_model.rs | 59 ++- .../src/metastore/retrying_metastore/mod.rs | 173 +++---- .../src/metastore/retrying_metastore/test.rs | 174 ++++--- quickwit/quickwit-metastore/src/tests.rs | 232 ++++++--- quickwit/quickwit-proto/build.rs | 8 +- .../protos/quickwit/metastore.proto | 101 ++-- .../codegen/quickwit/quickwit.metastore.rs | 377 ++++----------- .../quickwit-proto/src/control_plane/mod.rs | 2 + quickwit/quickwit-proto/src/indexing/mod.rs | 2 + quickwit/quickwit-proto/src/lib.rs | 102 +--- quickwit/quickwit-proto/src/metastore/mod.rs | 156 ++++++ quickwit/quickwit-proto/src/types.rs | 132 +++++ quickwit/quickwit-search/src/lib.rs | 27 +- quickwit/quickwit-search/src/root.rs | 40 +- .../quickwit-search/src/search_stream/root.rs | 24 +- 46 files changed, 2458 insertions(+), 2407 deletions(-) create mode 100644 quickwit/quickwit-proto/src/metastore/mod.rs create mode 100644 quickwit/quickwit-proto/src/types.rs diff --git a/quickwit/quickwit-indexing/Cargo.toml b/quickwit/quickwit-indexing/Cargo.toml index 77e7556d8ec..ee652ae98ac 100644 --- a/quickwit/quickwit-indexing/Cargo.toml +++ b/quickwit/quickwit-indexing/Cargo.toml @@ -73,6 +73,7 @@ testsuite = [ "quickwit-actors/testsuite", "quickwit-cluster/testsuite", "quickwit-common/testsuite", + "quickwit-metastore/testsuite", ] [dev-dependencies] diff --git a/quickwit/quickwit-indexing/src/actors/doc_processor.rs b/quickwit/quickwit-indexing/src/actors/doc_processor.rs index b5dd045a8ee..6000e4ca443 100644 --- a/quickwit/quickwit-indexing/src/actors/doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/doc_processor.rs @@ -27,6 +27,7 @@ use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, Qu use quickwit_common::runtimes::RuntimeType; use quickwit_config::{SourceInputFormat, TransformConfig}; use quickwit_doc_mapper::{DocMapper, DocParsingError, JsonObject}; +use quickwit_proto::IndexUid; use serde::Serialize; use serde_json::Value as JsonValue; use tantivy::schema::{Field, Value}; @@ -107,7 +108,7 @@ impl From for DocProcessorError { #[derive(Clone, Debug, Eq, PartialEq, Serialize)] pub struct DocProcessorCounters { - index_id: String, + index_uid: IndexUid, source_id: String, /// Overall number of documents received, partitioned /// into 4 categories: @@ -129,9 +130,9 @@ pub struct DocProcessorCounters { } impl DocProcessorCounters { - pub fn new(index_id: String, source_id: String) -> Self { + pub fn new(index_uid: IndexUid, source_id: String) -> Self { Self { - index_id, + index_uid, source_id, num_parse_errors: 0, num_transform_errors: 0, @@ -162,7 +163,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_docs_total .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "parsing_error", ]) @@ -170,7 +171,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_bytes .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "parsing_error", ]) @@ -183,7 +184,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_docs_total .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "transform_error", ]) @@ -191,7 +192,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_bytes .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "transform_error", ]) @@ -204,7 +205,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_docs_total .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "missing_field", ]) @@ -212,7 +213,7 @@ impl DocProcessorCounters { crate::metrics::INDEXER_METRICS .processed_bytes .with_label_values([ - self.index_id.as_str(), + self.index_uid.index_id(), self.source_id.as_str(), "missing_field", ]) @@ -224,11 +225,11 @@ impl DocProcessorCounters { self.overall_num_bytes += num_bytes; crate::metrics::INDEXER_METRICS .processed_docs_total - .with_label_values([self.index_id.as_str(), self.source_id.as_str(), "valid"]) + .with_label_values([self.index_uid.index_id(), self.source_id.as_str(), "valid"]) .inc(); crate::metrics::INDEXER_METRICS .processed_bytes - .with_label_values([self.index_id.as_str(), self.source_id.as_str(), "valid"]) + .with_label_values([self.index_uid.index_id(), self.source_id.as_str(), "valid"]) .inc_by(num_bytes); } } @@ -246,7 +247,7 @@ pub struct DocProcessor { impl DocProcessor { pub fn try_new( - index_id: String, + index_uid: IndexUid, source_id: String, doc_mapper: Arc, indexer_mailbox: Mailbox, @@ -261,7 +262,7 @@ impl DocProcessor { doc_mapper, indexer_mailbox, timestamp_field_opt, - counters: DocProcessorCounters::new(index_id, source_id), + counters: DocProcessorCounters::new(index_uid, source_id), publish_lock: PublishLock::default(), #[cfg(feature = "vrl")] transform_opt: transform_config_opt diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 8010a3b0cb9..9025d1b5105 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -30,8 +30,9 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_common::KillSwitch; use quickwit_config::{IndexingSettings, SourceConfig}; use quickwit_doc_mapper::DocMapper; -use quickwit_metastore::{Metastore, MetastoreError}; +use quickwit_metastore::{IndexMetadataResponseExt, Metastore, MetastoreError}; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::IndexMetadataRequest; use quickwit_storage::Storage; use tokio::join; use tokio::sync::Semaphore; @@ -254,11 +255,12 @@ impl IndexingPipeline { .await .expect("The semaphore should not be closed."); self.statistics.num_spawn_attempts += 1; - let index_id = self.params.pipeline_id.index_uid.index_id(); + let index_uid = self.params.pipeline_id.index_uid.clone(); + let index_id = index_uid.index_id(); let source_id = self.params.pipeline_id.source_id.as_str(); self.kill_switch = ctx.kill_switch().child(); info!( - index_id=%index_id, + index_id=%index_uid.index_id(), source_id=%source_id, pipeline_ord=%self.params.pipeline_id.pipeline_ord, root_dir=%self.params.indexing_directory.path().display(), @@ -351,7 +353,7 @@ impl IndexingPipeline { .spawn(indexer); let doc_processor = DocProcessor::try_new( - index_id.to_string(), + index_uid.clone(), source_id.to_string(), self.params.doc_mapper.clone(), indexer_mailbox, @@ -369,9 +371,11 @@ impl IndexingPipeline { .spawn(doc_processor); // Fetch index_metadata to be sure to have the last updated checkpoint. - let index_metadata = ctx - .protect_future(self.params.metastore.index_metadata(index_id)) + let index_metadata_request = IndexMetadataRequest::strict(index_uid.clone()); + let index_metadata_response = ctx + .protect_future(self.params.metastore.index_metadata(index_metadata_request)) .await?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let source_checkpoint = index_metadata .checkpoint .source_checkpoint(source_id) @@ -381,7 +385,7 @@ impl IndexingPipeline { .protect_future(quickwit_supported_sources().load_source( Arc::new(SourceExecutionContext { metastore: self.params.metastore.clone(), - index_uid: self.params.pipeline_id.index_uid.clone(), + index_uid, queues_dir_path: self.params.queues_dir_path.clone(), source_config: self.params.source_config.clone(), }), diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 34d12e1c248..c98555c07bb 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -37,11 +37,14 @@ use quickwit_config::{ build_doc_mapper, IndexConfig, IndexerConfig, SourceConfig, INGEST_API_SOURCE_ID, }; use quickwit_ingest::{DropQueueRequest, IngestApiService, ListQueuesRequest, QUEUES_DIR_NAME}; -use quickwit_metastore::{IndexMetadata, Metastore}; +use quickwit_metastore::{ + IndexMetadata, IndexMetadataResponseExt, ListIndexesResponseExt, Metastore, +}; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, ApplyIndexingPlanResponse, IndexingError, IndexingPipelineId, IndexingTask, }; +use quickwit_proto::metastore::{IndexMetadataRequest, ListIndexesRequest}; use quickwit_proto::IndexUid; use quickwit_storage::StorageResolver; use serde::{Deserialize, Serialize}; @@ -208,11 +211,11 @@ impl IndexingService { async fn spawn_pipeline( &mut self, ctx: &ActorContext, - index_id: String, + index_uid: IndexUid, source_config: SourceConfig, pipeline_ord: usize, ) -> Result { - let index_metadata = self.index_metadata(ctx, &index_id).await?; + let index_metadata = self.index_metadata(ctx, index_uid).await?; let pipeline_id = IndexingPipelineId { index_uid: index_metadata.index_uid.clone(), source_id: source_config.source_id.clone(), @@ -308,14 +311,18 @@ impl IndexingService { async fn index_metadata( &self, ctx: &ActorContext, - index_id: &str, + index_uid: IndexUid, ) -> Result { let _protect_guard = ctx.protect_zone(); - let index_metadata = self + let index_metadata_request = IndexMetadataRequest::strict(index_uid); + let index_metadata_response = self .metastore - .index_metadata(index_id) + .index_metadata(index_metadata_request) .await - .map_err(|err| IndexingError::MetastoreError(err.to_string()))?; + .map_err(|error| IndexingError::MetastoreError(error.to_string()))?; + let index_metadata = index_metadata_response + .deserialize_index_metadata() + .map_err(|error| IndexingError::MetastoreError(error.to_string()))?; Ok(index_metadata) } @@ -477,7 +484,7 @@ impl IndexingService { .iter() // No need to emit two request for the same `index_uid` .unique_by(|pipeline_id| pipeline_id.index_uid.clone()) - .map(|pipeline_id| self.index_metadata(ctx, pipeline_id.index_uid.index_id())); + .map(|pipeline_id| self.index_metadata(ctx, pipeline_id.index_uid.clone())); let indexes_metadata = try_join_all(indexes_metadata_futures).await?; let indexes_metadata_by_index_id: HashMap = indexes_metadata .into_iter() @@ -602,9 +609,10 @@ impl IndexingService { let index_ids: HashSet = self .metastore - .list_indexes_metadatas() + .list_indexes(ListIndexesRequest {}) .await .context("Failed to list queues")? + .deserialize_indexes_metadata()? .into_iter() .map(|index_metadata| index_metadata.index_id().to_string()) .collect(); @@ -714,7 +722,7 @@ impl Handler for IndexingService { Ok(self .spawn_pipeline( ctx, - message.index_id, + message.index_uid, message.source_config, message.pipeline_ord, ) @@ -775,8 +783,11 @@ mod tests { IngestApiConfig, SourceConfig, SourceInputFormat, SourceParams, VecSourceParams, }; use quickwit_ingest::{init_ingest_api, CreateQueueIfNotExistsRequest}; - use quickwit_metastore::{metastore_for_test, MockMetastore}; + use quickwit_metastore::{ + metastore_for_test, AddSourceRequestExt, CreateIndexRequestExt, MockMetastore, + }; use quickwit_proto::indexing::IndexingTask; + use quickwit_proto::{AddSourceRequest, CreateIndexRequest, DeleteIndexRequest}; use super::*; @@ -969,11 +980,16 @@ mod tests { let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); - metastore - .add_source(index_uid.clone(), SourceConfig::ingest_api_default()) - .await - .unwrap(); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_uid.clone(), index_config).unwrap(); + let create_index_response = metastore.create_index(create_index_request).await.unwrap(); + let index_uid: IndexUid = create_index_response.index_uid.into(); + + let source_config = SourceConfig::ingest_api_default(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); let (indexing_service, indexing_service_handle) = spawn_indexing_service( @@ -994,18 +1010,17 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - metastore - .add_source(index_uid.clone(), source_config_1.clone()) - .await - .unwrap(); - let metadata = metastore.index_metadata(index_id.as_str()).await.unwrap(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config_1).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + let indexing_tasks = vec![ IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: "test-indexing-service--source-1".to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: "test-indexing-service--source-1".to_string(), }, ]; @@ -1030,26 +1045,25 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - metastore - .add_source(index_uid.clone(), source_config_2.clone()) - .await - .unwrap(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config_2).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); let indexing_tasks = vec![ IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: INGEST_API_SOURCE_ID.to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: "test-indexing-service--source-1".to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: "test-indexing-service--source-1".to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: source_config_2.source_id.clone(), }, ]; @@ -1087,15 +1101,15 @@ mod tests { ); let indexing_tasks = vec![ IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: INGEST_API_SOURCE_ID.to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: "test-indexing-service--source-1".to_string(), }, IndexingTask { - index_uid: metadata.index_uid.to_string(), + index_uid: index_uid.clone().into(), source_id: source_config_2.source_id.clone(), }, ]; @@ -1130,15 +1144,17 @@ mod tests { HashSet::<_>::from_iter(self_member.indexing_tasks.iter()), HashSet::from_iter(indexing_tasks.iter()) ); - // Delete index and apply empty plan - metastore.delete_index(index_uid).await.unwrap(); + let delete_request = DeleteIndexRequest::new(index_uid); + metastore.delete_index(delete_request).await.unwrap(); + indexing_service .ask_for_res(ApplyIndexingPlanRequest { indexing_tasks: Vec::new(), }) .await .unwrap(); + let indexing_service_obs = indexing_service_handle.observe().await; assert_eq!(indexing_service_obs.num_running_pipelines, 0); assert_eq!(indexing_service_obs.num_deleted_queues, 1); @@ -1383,7 +1399,8 @@ mod tests { indexing_server.run_ingest_api_queues_gc().await.unwrap(); assert_eq!(indexing_server.counters.num_deleted_queues, 0); - metastore.delete_index(index_uid).await.unwrap(); + let delete_request = DeleteIndexRequest::new(index_uid); + metastore.delete_index(delete_request).await.unwrap(); indexing_server.run_ingest_api_queues_gc().await.unwrap(); assert_eq!(indexing_server.counters.num_deleted_queues, 1); diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index 754f00e0a25..ca2dfdcf37b 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -35,7 +35,7 @@ use quickwit_directories::UnionDirectory; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::{Metastore, SplitMetadata}; use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::metastore::DeleteTask; +use quickwit_proto::metastore::{DeleteTask, ListDeleteTasksRequest, MarkSplitsForDeletionRequest}; use quickwit_query::get_quickwit_fastfield_normalizer_manager; use quickwit_query::query_ast::QueryAst; use tantivy::directory::{DirectoryClone, MmapDirectory, RamDirectory}; @@ -331,12 +331,13 @@ impl MergeExecutor { merge_scratch_directory: TempDirectory, ctx: &ActorContext, ) -> anyhow::Result> { - let delete_tasks = ctx - .protect_future( - self.metastore - .list_delete_tasks(split.index_uid.clone(), split.delete_opstamp), - ) + let list_delete_tasks_request = + ListDeleteTasksRequest::new(split.index_uid.clone(), split.delete_opstamp); + let list_delete_tasks_response = ctx + .protect_future(self.metastore.list_delete_tasks(list_delete_tasks_request)) .await?; + let delete_tasks = list_delete_tasks_response.delete_tasks; + if delete_tasks.is_empty() { warn!( "No delete task found for split `{}` with `delete_optamp` = `{}`.", @@ -345,7 +346,6 @@ impl MergeExecutor { ); return Ok(None); } - let last_delete_opstamp = delete_tasks .iter() .map(|delete_task| delete_task.opstamp) @@ -355,7 +355,6 @@ impl MergeExecutor { delete_opstamp_start = split.delete_opstamp, num_delete_tasks = delete_tasks.len() ); - let (union_index_meta, split_directories) = open_split_directories(&tantivy_dirs, self.doc_mapper.tokenizer_manager())?; let controlled_directory = self @@ -386,8 +385,10 @@ impl MergeExecutor { "All documents from split `{}` were deleted.", split.split_id() ); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(split.index_uid, [split.split_id]); self.metastore - .mark_splits_for_deletion(split.index_uid.clone(), &[split.split_id()]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await?; return Ok(None); }; @@ -406,12 +407,11 @@ impl MergeExecutor { } else { None }; - let index_pipeline_id = IndexingPipelineId { - index_uid: split.index_uid, node_id: split.node_id.clone(), - pipeline_ord: 0, + index_uid: split.index_uid, source_id: split.source_id.clone(), + pipeline_ord: 0, }; let indexed_split = IndexedSplit { split_attrs: SplitAttrs { @@ -530,8 +530,9 @@ fn open_index>>( mod tests { use quickwit_actors::Universe; use quickwit_common::split_file; - use quickwit_metastore::SplitMetadata; + use quickwit_metastore::{SplitMetadata, StageSplitsRequestExt}; use quickwit_proto::metastore::DeleteQuery; + use quickwit_proto::{PublishSplitsRequest, StageSplitsRequest}; use serde_json::Value as JsonValue; use tantivy::{Inventory, ReloadPolicy}; @@ -706,17 +707,21 @@ mod tests { let mut new_split_metadata = split_metadata.clone(); new_split_metadata.split_id = new_split_id(); new_split_metadata.num_merge_ops = 1; + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + new_split_metadata.clone(), + ); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest::new( + index_uid.clone(), + [new_split_metadata.split_id()], + [split_metadata.split_id()], + None, + ); metastore - .stage_splits(index_uid.clone(), vec![new_split_metadata.clone()]) - .await - .unwrap(); - metastore - .publish_splits( - index_uid.clone(), - &[new_split_metadata.split_id()], - &[split_metadata.split_id()], - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); let expected_uncompressed_docs_size_in_bytes = diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 2b25cc2a881..be444259849 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -30,8 +30,12 @@ use quickwit_common::io::IoControls; use quickwit_common::temp_dir::TempDirectory; use quickwit_common::KillSwitch; use quickwit_doc_mapper::DocMapper; -use quickwit_metastore::{ListSplitsQuery, Metastore, MetastoreError, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, Metastore, MetastoreError, + SplitState, +}; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::ListSplitsRequest; use time::OffsetDateTime; use tokio::join; use tracing::{debug, error, info, instrument}; @@ -200,15 +204,17 @@ impl MergePipeline { merge_policy=?self.params.merge_policy, "Spawning merge pipeline.", ); - let query = ListSplitsQuery::for_index(self.params.pipeline_id.index_uid.clone()) + let list_splits_query = ListSplitsQuery::default() .with_split_state(SplitState::Published) .retain_immature(OffsetDateTime::now_utc()); - let published_splits = ctx - .protect_future(self.params.metastore.list_splits(query)) - .await? - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query( + self.params.pipeline_id.index_uid.clone(), + list_splits_query, + )?; + let list_splits_response = ctx + .protect_future(self.params.metastore.list_splits(list_splits_request)) + .await?; + let published_splits = list_splits_response.deserialize_splits_metadata()?; // Merge publisher let merge_publisher = Publisher::new( diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index aaaad7e1d8e..354c9e38e08 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -24,6 +24,7 @@ use async_trait::async_trait; use fail::fail_point; use quickwit_actors::{Actor, ActorContext, Handler, Mailbox, QueueCapacity}; use quickwit_metastore::Metastore; +use quickwit_proto::metastore::PublishSplitsRequest; use serde::Serialize; use tracing::{info, instrument}; @@ -121,29 +122,32 @@ impl Handler for Publisher { parent_span: _, } = split_update; - let split_ids: Vec<&str> = new_splits.iter().map(|split| split.split_id()).collect(); - - let replaced_split_ids_ref_vec: Vec<&str> = - replaced_split_ids.iter().map(String::as_str).collect(); + let new_split_ids: Vec = new_splits + .iter() + .map(|split| split.split_id().to_string()) + .collect(); if let Some(_guard) = publish_lock.acquire().await { - ctx.protect_future(self.metastore.publish_splits( + let publish_splits_request = PublishSplitsRequest::new( index_uid, - &split_ids[..], - &replaced_split_ids_ref_vec, - checkpoint_delta_opt.clone(), - )) - .await - .context("Failed to publish splits.")?; + &new_split_ids, + &replaced_split_ids, + checkpoint_delta_opt + .clone() + .map(|checkpoint_delta| checkpoint_delta.into()), + ); + ctx.protect_future(self.metastore.publish_splits(publish_splits_request)) + .await + .context("Failed to publish splits.")?; } else { // TODO: Remove the junk right away? info!( - split_ids=?split_ids, + split_ids=?new_split_ids, "Splits' publish lock is dead." ); return Ok(()); } - info!(new_splits=?split_ids, checkpoint_delta=?checkpoint_delta_opt, "publish-new-splits"); + info!(new_splits=?new_split_ids, checkpoint_delta=?checkpoint_delta_opt, "publish-new-splits"); if let Some(source_mailbox) = self.source_mailbox_opt.as_ref() { if let Some(checkpoint) = checkpoint_delta_opt { // We voluntarily do not log anything here. diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index bde305b28d6..e55992d22a5 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -30,7 +30,8 @@ use itertools::Itertools; use once_cell::sync::OnceCell; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, QueueCapacity}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; -use quickwit_metastore::{Metastore, SplitMetadata}; +use quickwit_metastore::{Metastore, SplitMetadata, StageSplitsRequestExt}; +use quickwit_proto::metastore::StageSplitsRequest; use quickwit_proto::IndexUid; use quickwit_storage::SplitPayloadBuilder; use serde::Serialize; @@ -300,7 +301,8 @@ impl Handler for Uploader { async move { fail_point!("uploader:intask:before"); - let mut split_metadata_list = Vec::with_capacity(batch.splits.len()); + let mut splits_metadata = Vec::with_capacity(batch.splits.len()); + for packaged_split in batch.splits.iter() { if batch.publish_lock.is_dead() { // TODO: Remove the junk right away? @@ -320,16 +322,18 @@ impl Handler for Uploader { split_streamer.footer_range.start..split_streamer.footer_range.end, ); - split_metadata_list.push(split_metadata); + splits_metadata.push(split_metadata); } + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata(index_uid.clone(), splits_metadata.clone())?; metastore - .stage_splits(index_uid.clone(), split_metadata_list.clone()) + .stage_splits(stage_splits_request) .await?; - counters.num_staged_splits.fetch_add(split_metadata_list.len() as u64, Ordering::SeqCst); + counters.num_staged_splits.fetch_add(splits_metadata.len() as u64, Ordering::Relaxed); let mut packaged_splits_and_metadata = Vec::with_capacity(batch.splits.len()); - for (packaged_split, metadata) in batch.splits.into_iter().zip(split_metadata_list) { + + for (packaged_split, metadata) in batch.splits.into_iter().zip(splits_metadata) { let upload_result = upload_split( &packaged_split, &metadata, diff --git a/quickwit/quickwit-indexing/src/models/indexing_service_message.rs b/quickwit/quickwit-indexing/src/models/indexing_service_message.rs index 4c2de58aabe..23ca4a5324f 100644 --- a/quickwit/quickwit-indexing/src/models/indexing_service_message.rs +++ b/quickwit/quickwit-indexing/src/models/indexing_service_message.rs @@ -19,12 +19,13 @@ use quickwit_config::SourceConfig; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::IndexUid; use crate::actors::MergePipelineId; #[derive(Clone, Debug)] pub struct SpawnPipeline { - pub index_id: String, + pub index_uid: IndexUid, pub source_config: SourceConfig, pub pipeline_ord: usize, } diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index f21d88847f5..e9bf62fdf05 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -32,6 +32,8 @@ use quickwit_config::KafkaSourceParams; use quickwit_metastore::checkpoint::{ PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, }; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::metastore::IndexMetadataRequest; use quickwit_proto::IndexUid; use rdkafka::config::{ClientConfig, RDKafkaLogLevel}; use rdkafka::consumer::{ @@ -345,12 +347,9 @@ impl KafkaSource { partitions: &[i32], assignment_tx: oneshot::Sender>, ) -> anyhow::Result<()> { - let index_metadata = ctx - .protect_future( - self.ctx - .metastore - .index_metadata_strict(&self.ctx.index_uid), - ) + let index_metadata_request = IndexMetadataRequest::strict(self.ctx.index_uid.clone()); + let index_metadata_response = ctx + .protect_future(self.ctx.metastore.index_metadata(index_metadata_request)) .await .with_context(|| { format!( @@ -358,6 +357,7 @@ impl KafkaSource { self.ctx.index_uid.index_id() ) })?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let checkpoint = index_metadata .checkpoint .source_checkpoint(&self.ctx.source_config.source_id) @@ -769,8 +769,9 @@ mod kafka_broker_tests { use quickwit_common::rand::append_random_suffix; use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; + use quickwit_metastore::metastore::StageSplitsRequestExt; use quickwit_metastore::{metastore_for_test, Metastore, SplitMetadata}; - use quickwit_proto::IndexUid; + use quickwit_proto::{IndexUid, PublishSplitsRequest, StageSplitsRequest}; use rdkafka::admin::{AdminClient, AdminOptions, NewTopic, TopicReplication}; use rdkafka::client::DefaultClientContext; use rdkafka::message::ToBytes; @@ -915,12 +916,11 @@ mod kafka_broker_tests { } let split_id = new_split_id(); let split_metadata = SplitMetadata::for_test(split_id.clone()); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata); + metastore.stage_splits(stage_splits_request).await.unwrap(); - let mut source_delta = SourceCheckpointDelta::default(); + let mut checkpoint_delta = SourceCheckpointDelta::default(); for (partition_id, from_position, to_position) in partition_deltas { source_delta .record_partition_delta( @@ -936,12 +936,10 @@ mod kafka_broker_tests { ) .unwrap(); } - let index_delta = IndexCheckpointDelta { - source_id: source_id.to_string(), - source_delta, - }; + let publish_splits_request = + PublishSplitsRequest::new(index_uid, &[split_id], &[], Some(checkpoint_delta)); metastore - .publish_splits(index_uid.clone(), &[&split_id], &[], Some(index_delta)) + .publish_splits(publish_splits_request) .await .unwrap(); index_uid diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 3715df6929c..bfbab782bdb 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -33,7 +33,10 @@ use quickwit_config::{ }; use quickwit_doc_mapper::DocMapper; use quickwit_ingest::{init_ingest_api, QUEUES_DIR_NAME}; -use quickwit_metastore::{Metastore, MetastoreResolver, Split, SplitMetadata, SplitState}; +use quickwit_metastore::{ + CreateIndexRequestExt, Metastore, MetastoreResolver, Split, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::CreateIndexRequest; use quickwit_proto::IndexUid; use quickwit_storage::{Storage, StorageResolver}; use serde_json::Value as JsonValue; @@ -97,7 +100,9 @@ impl TestSandbox { let metastore = metastore_resolver .resolve(&Uri::from_well_formed(METASTORE_URI)) .await?; - let index_uid = metastore.create_index(index_config.clone()).await?; + let create_index_request = CreateIndexRequest::try_from_index_config(index_config)?; + let create_index_response = metastore.create_index(create_index_request).await?; + let index_uid: IndexUid = create_index_response.index_uid.into(); let storage = storage_resolver.resolve(&index_uri).await?; let universe = Universe::with_accelerated_time(); let queues_dir_path = temp_dir.path().join(QUEUES_DIR_NAME); @@ -159,7 +164,7 @@ impl TestSandbox { let pipeline_id = self .indexing_service .ask_for_res(SpawnPipeline { - index_id: self.index_uid.index_id().to_string(), + index_uid: self.index_uid.clone(), source_config, pipeline_ord: 0, }) diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 13c442e2017..d944eb306dc 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -34,8 +34,9 @@ use quickwit_indexing::actors::{ }; use quickwit_indexing::merge_policy::merge_policy_from_settings; use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; -use quickwit_metastore::Metastore; +use quickwit_metastore::{IndexMetadataResponseExt, Metastore}; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::IndexMetadataRequest; use quickwit_proto::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::Storage; @@ -148,10 +149,12 @@ impl DeleteTaskPipeline { root_dir=%self.delete_service_task_dir.to_str().unwrap(), "Spawning delete tasks pipeline.", ); - let index_metadata = self + let index_metadata_request = IndexMetadataRequest::strict(self.index_uid.clone()); + let index_metadata_response = self .metastore - .index_metadata_strict(&self.index_uid) + .index_metadata(index_metadata_request) .await?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let index_config = index_metadata.into_index_config(); let publisher = Publisher::new( PublisherType::MergePublisher, diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index d775a61cb5e..76075956f40 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -30,13 +30,17 @@ use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; use quickwit_indexing::actors::MergeSplitDownloader; use quickwit_indexing::merge_policy::MergeOperation; use quickwit_metastore::{ - split_tag_filter, split_time_range_filter, Metastore, MetastoreResult, Split, + split_tag_filter, split_time_range_filter, ListSplitsQuery, ListSplitsRequestExt, + ListSplitsResponseExt, Metastore, MetastoreResult, Split, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::{ + DeleteTask, ListDeleteTasksRequest, ListSplitsRequest, UpdateSplitsDeleteOpstampRequest, }; -use quickwit_proto::metastore::DeleteTask; use quickwit_proto::{IndexUid, SearchRequest}; use quickwit_search::{jobs_to_leaf_request, SearchJob, SearchJobPlacer}; use serde::Serialize; use tantivy::Inventory; +use time::OffsetDateTime; use tracing::{debug, info}; use crate::metrics::JANITOR_METRICS; @@ -159,7 +163,7 @@ impl DeleteTaskPlanner { } let (splits_with_deletes, splits_without_deletes) = - self.partition_splits_by_deletes(&stale_splits, ctx).await?; + self.partition_splits_by_deletes(stale_splits, ctx).await?; info!( "{} splits with deletes, {} splits without deletes.", @@ -173,18 +177,21 @@ impl DeleteTaskPlanner { .iter() .map(|split| split.split_id()) .collect_vec(); - ctx.protect_future(self.metastore.update_splits_delete_opstamp( + let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest::new( self.index_uid.clone(), - &split_ids_without_delete, + split_ids_without_delete, last_delete_opstamp, - )) + ); + ctx.protect_future( + self.metastore + .update_splits_delete_opstamp(update_splits_delete_opstamp_request), + ) .await?; // Sends delete operations. for split_with_deletes in splits_with_deletes { - let delete_operation = MergeOperation::new_delete_and_merge_operation( - split_with_deletes.split_metadata, - ); + let delete_operation = + MergeOperation::new_delete_and_merge_operation(split_with_deletes); info!(delete_operation=?delete_operation, "Planned delete operation."); let tracked_delete_operation = self .ongoing_delete_operations_inventory @@ -200,7 +207,6 @@ impl DeleteTaskPlanner { .set(self.ongoing_delete_operations_inventory.list().len() as i64); } } - Ok(()) } @@ -208,19 +214,19 @@ impl DeleteTaskPlanner { /// splits that do not and returns the two groups. async fn partition_splits_by_deletes( &self, - stale_splits: &[Split], + stale_splits: Vec, ctx: &ActorContext, - ) -> anyhow::Result<(Vec, Vec)> { - let mut splits_without_deletes: Vec = Vec::new(); - let mut splits_with_deletes: Vec = Vec::new(); + ) -> anyhow::Result<(Vec, Vec)> { + let mut splits_without_deletes: Vec = Vec::new(); + let mut splits_with_deletes: Vec = Vec::new(); for stale_split in stale_splits { + let list_delete_tasks_request = + ListDeleteTasksRequest::new(self.index_uid.clone(), stale_split.delete_opstamp); let pending_tasks = ctx - .protect_future(self.metastore.list_delete_tasks( - self.index_uid.clone(), - stale_split.split_metadata.delete_opstamp, - )) - .await?; + .protect_future(self.metastore.list_delete_tasks(list_delete_tasks_request)) + .await? + .delete_tasks; // Keep only delete tasks that matches the split metadata. let pending_and_matching_metadata_tasks = pending_tasks @@ -238,8 +244,8 @@ impl DeleteTaskPlanner { let delete_query_ast = serde_json::from_str(&delete_query.query_ast) .expect("Failed to deserialize query_ast json"); let tags_filter = extract_tags_from_query(delete_query_ast); - split_time_range_filter(stale_split, time_range.as_ref()) - && split_tag_filter(stale_split, tags_filter.as_ref()) + split_time_range_filter(&stale_split, time_range.as_ref()) + && split_tag_filter(&stale_split, tags_filter.as_ref()) }) .collect_vec(); @@ -252,7 +258,7 @@ impl DeleteTaskPlanner { let has_split_docs_to_delete = self .has_split_docs_to_delete( - stale_split, + &stale_split, &pending_and_matching_metadata_tasks, &self.doc_mapper_str, self.index_uri.as_str(), @@ -262,12 +268,11 @@ impl DeleteTaskPlanner { ctx.record_progress(); if has_split_docs_to_delete { - splits_with_deletes.push(stale_split.clone()); + splits_with_deletes.push(stale_split); } else { - splits_without_deletes.push(stale_split.clone()); + splits_without_deletes.push(stale_split); } } - Ok((splits_with_deletes, splits_without_deletes)) } @@ -275,13 +280,13 @@ impl DeleteTaskPlanner { /// if it matches documents. async fn has_split_docs_to_delete( &self, - stale_split: &Split, + stale_split: &SplitMetadata, delete_tasks: &[DeleteTask], doc_mapper_str: &str, index_uri: &str, ctx: &ActorContext, ) -> anyhow::Result { - let search_job = SearchJob::from(&stale_split.split_metadata); + let search_job = SearchJob::from(stale_split); let mut search_client = self .search_job_placer .assign_job(search_job.clone(), &HashSet::new()) @@ -290,11 +295,11 @@ impl DeleteTaskPlanner { let delete_query = delete_task .delete_query .as_ref() - .expect("Delete task must have a delete query."); + .expect("Delete task should have a delete query."); + let index_uid: IndexUid = delete_query.index_uid.clone().into(); + let index_id = index_uid.index_id().to_string(); let search_request = SearchRequest { - index_id: IndexUid::from(delete_query.index_uid.clone()) - .index_id() - .to_string(), + index_id, query_ast: delete_query.query_ast.clone(), start_timestamp: delete_query.start_timestamp, end_timestamp: delete_query.end_timestamp, @@ -322,18 +327,23 @@ impl DeleteTaskPlanner { index_uid: IndexUid, last_delete_opstamp: u64, ctx: &ActorContext, - ) -> MetastoreResult> { - let stale_splits = ctx - .protect_future(self.metastore.list_stale_splits( - index_uid.clone(), - last_delete_opstamp, - NUM_STALE_SPLITS_TO_FETCH, - )) + ) -> MetastoreResult> { + let list_splits_query = ListSplitsQuery::default() + .with_delete_opstamp_lt(last_delete_opstamp) + .with_split_state(SplitState::Published) + .with_limit(NUM_STALE_SPLITS_TO_FETCH) + .retain_mature(OffsetDateTime::now_utc()) + .sort_by_staleness(); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid.clone(), list_splits_query)?; + let list_splits_response = ctx + .protect_future(self.metastore.list_splits(list_splits_request)) .await?; + let stale_splits = list_splits_response.deserialize_splits_metadata()?; debug!( index_id = index_uid.index_id(), last_delete_opstamp = last_delete_opstamp, - num_stale_splits_from_metastore = stale_splits.len() + num_stale_splits = stale_splits.len() ); let ongoing_delete_operations = self.ongoing_delete_operations_inventory.list(); let filtered_splits = stale_splits @@ -348,7 +358,7 @@ impl DeleteTaskPlanner { == stale_split.split_id() }) }) - .collect_vec(); + .collect(); Ok(filtered_splits) } } diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs index 8102e5256a6..361ae656939 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs @@ -26,7 +26,8 @@ use async_trait::async_trait; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, ActorHandle, Handler}; use quickwit_common::temp_dir::{self}; use quickwit_config::IndexConfig; -use quickwit_metastore::Metastore; +use quickwit_metastore::{ListIndexesResponseExt, Metastore}; +use quickwit_proto::metastore::ListIndexesRequest; use quickwit_proto::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::StorageResolver; @@ -108,8 +109,9 @@ impl DeleteTaskService { ) -> anyhow::Result<()> { let mut index_config_by_index_id: HashMap = self .metastore - .list_indexes_metadatas() + .list_indexes(ListIndexesRequest {}) .await? + .deserialize_indexes_metadata()? .into_iter() .map(|index_metadata| { ( @@ -140,8 +142,12 @@ impl DeleteTaskService { for index_uid in index_uids.difference(&pipeline_index_uids) { let index_config = index_config_by_index_id .remove(index_uid) - .expect("Index metadata must be present."); - if self.spawn_pipeline(index_config, ctx).await.is_err() { + .expect("Index metadata should be present."); + if self + .spawn_pipeline(index_uid.clone(), index_config, ctx) + .await + .is_err() + { warn!( "Failed to spawn delete pipeline for {}", index_uid.index_id() @@ -154,17 +160,14 @@ impl DeleteTaskService { pub async fn spawn_pipeline( &mut self, + index_uid: IndexUid, index_config: IndexConfig, ctx: &ActorContext, ) -> anyhow::Result<()> { let index_uri = index_config.index_uri.clone(); let index_storage = self.storage_resolver.resolve(&index_uri).await?; - let index_metadata = self - .metastore - .index_metadata(index_config.index_id.as_str()) - .await?; let pipeline = DeleteTaskPipeline::new( - index_metadata.index_uid.clone(), + index_uid.clone(), self.metastore.clone(), self.search_job_placer.clone(), index_storage, @@ -173,7 +176,7 @@ impl DeleteTaskService { ); let (_pipeline_mailbox, pipeline_handler) = ctx.spawn_actor().spawn(pipeline); self.pipeline_handles_by_index_uid - .insert(index_metadata.index_uid, pipeline_handler); + .insert(index_uid, pipeline_handler); Ok(()) } } diff --git a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs index ddfe2b79504..7f9e5130225 100644 --- a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs +++ b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs @@ -26,7 +26,8 @@ use async_trait::async_trait; use futures::{stream, StreamExt}; use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, Handler}; -use quickwit_metastore::Metastore; +use quickwit_metastore::{ListIndexesResponseExt, Metastore}; +use quickwit_proto::metastore::ListIndexesRequest; use quickwit_storage::StorageResolver; use serde::Serialize; use tracing::{error, info}; @@ -93,39 +94,55 @@ impl GarbageCollector { info!("garbage-collect-operation"); self.counters.num_passes += 1; - let indexes = match self.metastore.list_indexes_metadatas().await { - Ok(metadatas) => metadatas, + let list_indexes_response = match self.metastore.list_indexes(ListIndexesRequest {}).await { + Ok(list_indexes_response) => list_indexes_response, Err(error) => { - error!(error=?error, "Failed to list indexes from the metastore."); + error!(error=?error, "Failed to list indexes from metastore."); return; } }; - info!(index_ids=%indexes.iter().map(|im| im.index_id()).join(", "), "Garbage collecting indexes."); - - let mut gc_futures = stream::iter(indexes).map(|index| { - let metastore = self.metastore.clone(); - let storage_resolver = self.storage_resolver.clone(); - async move { - let index_uri = index.index_uri(); - let storage = match storage_resolver.resolve(index_uri).await { - Ok(storage) => storage, - Err(error) => { - error!(index=%index.index_id(), error=?error, "Failed to resolve the index storage Uri."); - return None; + let indexes_metadata = match list_indexes_response.deserialize_indexes_metadata() { + Ok(indexes_metadata) => indexes_metadata, + Err(error) => { + error!(error=?error, "Failed to deserialize indexes metadata."); + return; + } + }; + info!(index_ids=%indexes_metadata.iter().map(|index_metadata| index_metadata.index_id()).join(", "), "Garbage collecting indexes."); + + let mut gc_futures = stream::iter(indexes_metadata) + .map(|index_metadata| { + let metastore = self.metastore.clone(); + let storage_resolver = self.storage_resolver.clone(); + async move { + let index_uri = index_metadata.index_uri(); + let storage = match storage_resolver.resolve(index_uri).await { + Ok(storage) => storage, + Err(error) => { + error!( + index_id=%index_metadata.index_id(), + index_uri=%index_uri, + error=?error, + "Failed to resolve index storage URI." + ); + return None; + } + }; + let index_uid = index_metadata.index_uid; + let gc_res = run_garbage_collect( + index_uid.clone(), + storage, + metastore, + STAGED_GRACE_PERIOD, + DELETION_GRACE_PERIOD, + false, + Some(ctx), + ) + .await; + Some((index_uid, gc_res)) } - }; - let index_uid = index.index_uid; - let gc_res = run_garbage_collect( - index_uid.clone(), - storage, - metastore, - STAGED_GRACE_PERIOD, - DELETION_GRACE_PERIOD, - false, - Some(ctx), - ).await; - Some((index_uid, gc_res)) - }}).buffer_unordered(MAX_CONCURRENT_GC_TASKS); + }) + .buffer_unordered(MAX_CONCURRENT_GC_TASKS); while let Some(gc_future_res) = gc_futures.next().await { let Some((index_uid, gc_res)) = gc_future_res else { diff --git a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs index e25638bec84..5536f01add7 100644 --- a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs +++ b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs @@ -25,7 +25,8 @@ use async_trait::async_trait; use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, Handler}; use quickwit_config::IndexConfig; -use quickwit_metastore::Metastore; +use quickwit_metastore::{ListIndexesResponseExt, Metastore}; +use quickwit_proto::metastore::ListIndexesRequest; use quickwit_proto::IndexUid; use serde::Serialize; use tracing::{debug, error, info}; @@ -81,18 +82,25 @@ impl RetentionPolicyExecutor { debug!("retention-policy-refresh-indexes-operation"); self.counters.num_refresh_passes += 1; - let index_metadatas = match self.metastore.list_indexes_metadatas().await { - Ok(metadatas) => metadatas, + let list_indexes_response = match self.metastore.list_indexes(ListIndexesRequest {}).await { + Ok(list_indexes_response) => list_indexes_response, Err(error) => { - error!(error=?error, "Failed to list indexes from the metastore."); + error!(error=?error, "Failed to list indexes from metastore."); return; } }; - debug!(index_ids=%index_metadatas.iter().map(|im| im.index_id()).join(", "), "Retention policy refresh."); + let indexes_metadata = match list_indexes_response.deserialize_indexes_metadata() { + Ok(indexes_metadata) => indexes_metadata, + Err(error) => { + error!(error=?error, "Failed to deserialize indexes metadata."); + return; + } + }; + debug!(index_ids=%indexes_metadata.iter().map(|index_metadata| index_metadata.index_id()).join(", "), "Refresh retention policies."); let deleted_indexes = compute_deleted_indexes( self.index_configs.keys().map(String::as_str), - index_metadatas + indexes_metadata .iter() .map(|index_metadata| index_metadata.index_id()), ); @@ -102,8 +110,7 @@ impl RetentionPolicyExecutor { self.index_configs.remove(&index_id); } } - - for index_metadata in index_metadatas { + for index_metadata in indexes_metadata { let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); // We only care about indexes with a retention policy configured. diff --git a/quickwit/quickwit-janitor/src/garbage_collection.rs b/quickwit/quickwit-janitor/src/garbage_collection.rs index 3372de1d5fe..2f6fc674a11 100644 --- a/quickwit/quickwit-janitor/src/garbage_collection.rs +++ b/quickwit/quickwit-janitor/src/garbage_collection.rs @@ -26,7 +26,11 @@ use futures::Future; use quickwit_actors::ActorContext; use quickwit_common::PrettySample; use quickwit_metastore::{ - ListSplitsQuery, Metastore, MetastoreError, SplitInfo, SplitMetadata, SplitState, + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, Metastore, MetastoreError, + SplitInfo, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::{ + DeleteSplitsRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, }; use quickwit_proto::IndexUid; use quickwit_storage::{BulkDeleteError, Storage}; @@ -97,31 +101,29 @@ pub async fn run_garbage_collect( let grace_period_timestamp = OffsetDateTime::now_utc().unix_timestamp() - staged_grace_period.as_secs() as i64; - let query = ListSplitsQuery::for_index(index_uid.clone()) + let list_splits_query = ListSplitsQuery::default() .with_split_state(SplitState::Staged) .with_update_timestamp_lte(grace_period_timestamp); - - let deletable_staged_splits: Vec = - protect_future(ctx_opt, metastore.list_splits(query)) - .await? - .into_iter() - .map(|meta| meta.split_metadata) - .collect(); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid.clone(), list_splits_query)?; + let list_splits_response = + protect_future(ctx_opt, metastore.list_splits(list_splits_request)).await?; + let deletable_staged_splits = list_splits_response.deserialize_splits_metadata()?; if dry_run { - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::MarkedForDeletion); + let list_splits_query = + ListSplitsQuery::default().with_split_state(SplitState::MarkedForDeletion); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid.clone(), list_splits_query)?; + let list_splits_response = + protect_future(ctx_opt, metastore.list_splits(list_splits_request)).await?; + let mut splits_marked_for_deletion = list_splits_response.deserialize_splits_metadata()?; - let mut splits_marked_for_deletion = protect_future(ctx_opt, metastore.list_splits(query)) - .await? - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); splits_marked_for_deletion.extend(deletable_staged_splits); let candidate_entries: Vec = splits_marked_for_deletion .into_iter() - .map(|split| split.as_split_info()) + .map(|split_metadata| split_metadata.as_split_info()) .collect(); return Ok(SplitRemovalInfo { removed_split_entries: candidate_entries, @@ -130,18 +132,18 @@ pub async fn run_garbage_collect( } // Schedule all eligible staged splits for delete - let split_ids: Vec<&str> = deletable_staged_splits - .iter() - .map(|split| split.split_id()) + let split_ids: Vec = deletable_staged_splits + .into_iter() + .map(|split_metadata| split_metadata.split_id) .collect(); if !split_ids.is_empty() { + let mark_splits_request = MarkSplitsForDeletionRequest::new(index_uid.clone(), split_ids); protect_future( ctx_opt, - metastore.mark_splits_for_deletion(index_uid.clone(), &split_ids), + metastore.mark_splits_for_deletion(mark_splits_request), ) .await?; } - // We delete splits marked for deletion that have an update timestamp anterior // to `now - deletion_grace_period`. let updated_before_timestamp = @@ -176,25 +178,38 @@ async fn delete_splits_marked_for_deletion( let mut failed_splits = Vec::new(); loop { - let query = ListSplitsQuery::for_index(index_uid.clone()) + let list_splits_query = ListSplitsQuery::default() .with_split_state(SplitState::MarkedForDeletion) .with_update_timestamp_lte(updated_before_timestamp) .with_limit(DELETE_SPLITS_BATCH_SIZE); - let list_splits_result = protect_future(ctx_opt, metastore.list_splits(query)).await; - - let splits_to_delete = match list_splits_result { - Ok(splits) => splits, + let list_splits_request = match ListSplitsRequest::try_from_list_splits_query( + index_uid.clone(), + list_splits_query, + ) { + Ok(list_splits_request) => list_splits_request, Err(error) => { - error!(error = ?error, "Failed to fetch deletable splits."); + error!(error=?error, "Failed to serialize list splits query."); break; } }; - let splits_to_delete = splits_to_delete - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); + let list_splits_result = + protect_future(ctx_opt, metastore.list_splits(list_splits_request)).await; + let list_splits_response = match list_splits_result { + Ok(list_splits_response) => list_splits_response, + Err(error) => { + error!(error=?error, "Failed to list splits marked for deletion."); + break; + } + }; + let splits_to_delete = match list_splits_response.deserialize_splits_metadata() { + Ok(splits_metadata) => splits_metadata, + Err(error) => { + error!(error=?error, "Failed to deserialize splits metadata."); + break; + } + }; let num_splits_to_delete = splits_to_delete.len(); if num_splits_to_delete == 0 { @@ -287,22 +302,24 @@ pub async fn delete_splits_from_storage_and_metastore( } }; if !successes.is_empty() { - let split_ids: Vec<&str> = successes + let split_ids: Vec = successes .iter() - .map(|split_info| split_info.split_id.as_str()) + .map(|split_info| split_info.split_id.clone()) .collect(); - let metastore_result = protect_future( - ctx_opt, - metastore.delete_splits(index_uid.clone(), &split_ids), - ) - .await; + let delete_splits_request = DeleteSplitsRequest::new(index_uid.clone(), split_ids); + let metastore_result = + protect_future(ctx_opt, metastore.delete_splits(delete_splits_request)).await; if let Err(metastore_error) = metastore_result { + let failed_split_ids: Vec<&String> = successes + .iter() + .map(|split_info| &split_info.split_id) + .collect(); error!( error=?metastore_error, index_id=index_uid.index_id(), "Failed to delete split(s) {:?} from metastore.", - PrettySample::new(&split_ids, 5), + PrettySample::new(&failed_split_ids, 5), ); let delete_splits_error = DeleteSplitsError { successes: Vec::new(), diff --git a/quickwit/quickwit-janitor/src/retention_policy_execution.rs b/quickwit/quickwit-janitor/src/retention_policy_execution.rs index 972af691f90..75b0dd4bd04 100644 --- a/quickwit/quickwit-janitor/src/retention_policy_execution.rs +++ b/quickwit/quickwit-janitor/src/retention_policy_execution.rs @@ -22,7 +22,11 @@ use std::sync::Arc; use quickwit_actors::ActorContext; use quickwit_common::PrettySample; use quickwit_config::RetentionPolicy; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitMetadata, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, Metastore, SplitMetadata, + SplitState, +}; +use quickwit_proto::metastore::{ListSplitsRequest, MarkSplitsForDeletionRequest}; use quickwit_proto::IndexUid; use time::OffsetDateTime; use tracing::{info, warn}; @@ -47,16 +51,20 @@ pub async fn run_execute_retention_policy( let retention_period = retention_policy.retention_period()?; let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); let max_retention_timestamp = current_timestamp - retention_period.as_secs() as i64; - let query = ListSplitsQuery::for_index(index_uid.clone()) + let list_splits_query = ListSplitsQuery::default() .with_split_state(SplitState::Published) .with_time_range_end_lte(max_retention_timestamp); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid.clone(), list_splits_query)?; + let list_splits_response = ctx + .protect_future(metastore.list_splits(list_splits_request)) + .await?; - let (expired_splits, ignored_splits): (Vec, Vec) = ctx - .protect_future(metastore.list_splits(query)) - .await? - .into_iter() - .map(|split| split.split_metadata) - .partition(|split_metadata| split_metadata.time_range.is_some()); + let (expired_splits, ignored_splits): (Vec, Vec) = + list_splits_response + .deserialize_splits_metadata()? + .into_iter() + .partition(|split_metadata| split_metadata.time_range.is_some()); if !ignored_splits.is_empty() { let ignored_split_ids: Vec = ignored_splits @@ -74,9 +82,9 @@ pub async fn run_execute_retention_policy( return Ok(expired_splits); } // Mark the expired splits for deletion. - let expired_split_ids: Vec<&str> = expired_splits + let expired_split_ids: Vec = expired_splits .iter() - .map(|split_metadata| split_metadata.split_id()) + .map(|split_metadata| split_metadata.split_id.clone()) .collect(); info!( index_id=%index_uid.index_id(), @@ -84,7 +92,8 @@ pub async fn run_execute_retention_policy( "Marking {} splits for deletion based on retention policy.", expired_split_ids.len() ); - ctx.protect_future(metastore.mark_splits_for_deletion(index_uid, &expired_split_ids)) + let mark_splits_request = MarkSplitsForDeletionRequest::new(index_uid, expired_split_ids); + ctx.protect_future(metastore.mark_splits_for_deletion(mark_splits_request)) .await?; Ok(expired_splits) } diff --git a/quickwit/quickwit-metastore-utils/src/bin/proxy.rs b/quickwit/quickwit-metastore-utils/src/bin/proxy.rs index 57ffb14fa64..2bf1c10bd6c 100644 --- a/quickwit/quickwit-metastore-utils/src/bin/proxy.rs +++ b/quickwit/quickwit-metastore-utils/src/bin/proxy.rs @@ -142,7 +142,7 @@ impl MetastoreService for MetastoreProxyService { async fn stage_splits( &self, request: Request, - ) -> Result, Status> { + ) -> Result, Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.stage_splits(request).await?; @@ -152,7 +152,7 @@ impl MetastoreService for MetastoreProxyService { async fn publish_splits( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.publish_splits(request).await?; @@ -162,7 +162,7 @@ impl MetastoreService for MetastoreProxyService { async fn mark_splits_for_deletion( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.mark_splits_for_deletion(request).await?; @@ -172,7 +172,7 @@ impl MetastoreService for MetastoreProxyService { async fn delete_splits( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.delete_splits(request).await?; @@ -182,7 +182,7 @@ impl MetastoreService for MetastoreProxyService { async fn add_source( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.add_source(request).await?; @@ -192,7 +192,7 @@ impl MetastoreService for MetastoreProxyService { async fn toggle_source( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.toggle_source(request).await?; @@ -202,7 +202,7 @@ impl MetastoreService for MetastoreProxyService { async fn delete_source( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.delete_source(request).await?; @@ -212,7 +212,7 @@ impl MetastoreService for MetastoreProxyService { async fn reset_source_checkpoint( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); let resp = lock.client.reset_source_checkpoint(request).await?; diff --git a/quickwit/quickwit-metastore/src/checkpoint.rs b/quickwit/quickwit-metastore/src/checkpoint.rs index 223590c2734..1fd83f391dd 100644 --- a/quickwit/quickwit-metastore/src/checkpoint.rs +++ b/quickwit/quickwit-metastore/src/checkpoint.rs @@ -19,12 +19,13 @@ use std::cmp::Ordering; use std::collections::btree_map::Entry; -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::fmt; use std::iter::FromIterator; use std::ops::Range; use std::sync::Arc; +use quickwit_proto::metastore::SourceCheckpointDelta as ProtoSourceCheckpointDelta; use serde::ser::SerializeMap; use serde::{Deserialize, Serialize}; use thiserror::Error; @@ -382,6 +383,7 @@ impl fmt::Debug for SourceCheckpoint { } } +// TODO: Remove `PartitionDelta` and generalize use of `quickwit_proto::PartitionDelta` instead. /// A partition delta represents an interval (from, to] over a partition of a source. #[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] struct PartitionDelta { @@ -389,6 +391,8 @@ struct PartitionDelta { pub to: Position, } +// TODO: Remove `SourceCheckpointDelta` and generalize use of `quickwit_proto::PartitionDelta` +// instead. /// A checkpoint delta represents a checkpoint update. /// /// It is shipped as part of a split to convey the update @@ -404,6 +408,8 @@ pub struct SourceCheckpointDelta { per_partition: BTreeMap, } +// TODO: Remove `IndexCheckpointDelta` and generalize use of `quickwit_proto::SourceCheckpointDelta` +// instead. #[derive(Clone, PartialEq, Eq, Serialize, Deserialize)] pub struct IndexCheckpointDelta { pub source_id: String, @@ -431,6 +437,43 @@ impl fmt::Debug for IndexCheckpointDelta { } } +impl From for IndexCheckpointDelta { + fn from(delta: ProtoSourceCheckpointDelta) -> Self { + let mut per_partition = BTreeMap::new(); + + for (partition_id, partition_delta) in delta.partition_deltas { + let partition_id = PartitionId::from(partition_id); + let partition_delta = PartitionDelta { + from: Position::from(partition_delta.from_position_exclusive), + to: Position::from(partition_delta.to_position_inclusive), + }; + per_partition.insert(partition_id, partition_delta); + } + let source_delta = SourceCheckpointDelta { per_partition }; + Self { + source_id: delta.source_id, + source_delta, + } + } +} + +impl Into for IndexCheckpointDelta { + fn into(self) -> ProtoSourceCheckpointDelta { + let mut partition_deltas = HashMap::with_capacity(self.source_delta.per_partition.len()); + for (partition_id, partition_delta) in self.source_delta.per_partition { + let partition_delta = quickwit_proto::metastore::PartitionDelta { + from_position_exclusive: partition_delta.from.as_str().to_string(), + to_position_inclusive: partition_delta.to.as_str().to_string(), + }; + partition_deltas.insert(partition_id.0.to_string(), partition_delta); + } + ProtoSourceCheckpointDelta { + source_id: self.source_id, + partition_deltas, + } + } +} + impl fmt::Debug for SourceCheckpointDelta { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { f.write_str("∆(")?; diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index 788fea5df8f..a00fe5bcfea 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -51,7 +51,11 @@ pub use metastore::postgresql_metastore::PostgresqlMetastore; pub use metastore::retrying_metastore::RetryingMetastore; #[cfg(any(test, feature = "testsuite"))] pub use metastore::MockMetastore; -pub use metastore::{file_backed_metastore, IndexMetadata, ListSplitsQuery, Metastore}; +pub use metastore::{ + file_backed_metastore, AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, + IndexMetadataResponseExt, ListIndexesResponseExt, ListSplitsQuery, ListSplitsRequestExt, + ListSplitsResponseExt, Metastore, StageSplitsRequestExt, +}; pub use metastore_factory::{MetastoreFactory, UnsupportedMetastore}; pub use metastore_resolver::MetastoreResolver; use quickwit_common::is_disjoint; @@ -73,8 +77,11 @@ pub struct MetastoreApiSchemas; /// Returns `true` if the split time range is included in `time_range_opt`. /// If `time_range_opt` is None, returns always true. -pub fn split_time_range_filter(split: &Split, time_range_opt: Option<&Range>) -> bool { - match (time_range_opt, split.split_metadata.time_range.as_ref()) { +pub fn split_time_range_filter( + split_metadata: &SplitMetadata, + time_range_opt: Option<&Range>, +) -> bool { + match (time_range_opt, split_metadata.time_range.as_ref()) { (Some(filter_time_range), Some(split_time_range)) => { !is_disjoint(filter_time_range, split_time_range) } @@ -84,9 +91,12 @@ pub fn split_time_range_filter(split: &Split, time_range_opt: Option<&Range /// Returns `true` if the tags filter evaluation is true. /// If `tags_filter_opt` is None, returns always true. -pub fn split_tag_filter(split: &Split, tags_filter_opt: Option<&TagFilterAst>) -> bool { +pub fn split_tag_filter( + split_metadata: &SplitMetadata, + tags_filter_opt: Option<&TagFilterAst>, +) -> bool { tags_filter_opt - .map(|tags_filter_ast| tags_filter_ast.evaluate(&split.split_metadata.tags)) + .map(|tags_filter_ast| tags_filter_ast.evaluate(&split_metadata.tags)) .unwrap_or(true) } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs index a91308367f7..2452ea3da3e 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs @@ -27,16 +27,16 @@ use std::collections::HashMap; use std::fmt::Debug; use std::ops::Bound; +use itertools::Itertools; use quickwit_common::PrettySample; use quickwit_config::SourceConfig; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{DeleteQuery, DeleteTask, SourceCheckpointDelta}; use quickwit_proto::IndexUid; use serde::{Deserialize, Serialize}; use serialize::VersionedFileBackedIndex; use time::OffsetDateTime; use tracing::{info, warn}; -use crate::checkpoint::IndexCheckpointDelta; use crate::{ split_tag_filter, IndexMetadata, ListSplitsQuery, MetastoreError, MetastoreResult, Split, SplitMetadata, SplitState, @@ -210,8 +210,8 @@ impl FileBackedIndex { /// Marks the splits for deletion. Returns whether a mutation occurred. pub(crate) fn mark_splits_for_deletion( &mut self, - split_ids: &[&str], - deletable_states: &[SplitState], + split_ids: impl IntoIterator>, + deletable_split_states: &[SplitState], return_error_on_splits_not_found: bool, ) -> MetastoreResult { let mut mutation_occurred = false; @@ -219,24 +219,24 @@ impl FileBackedIndex { let mut non_deletable_split_ids = Vec::new(); let now_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - for &split_id in split_ids { + for split_id in split_ids { + let split_id_ref = split_id.as_ref(); // Check for the existence of split. - let metadata = match self.splits.get_mut(split_id) { + let metadata = match self.splits.get_mut(split_id_ref) { Some(metadata) => metadata, None => { - split_not_found_ids.push(split_id.to_string()); + split_not_found_ids.push(split_id_ref.to_string()); continue; } }; - if !deletable_states.contains(&metadata.split_state) { - non_deletable_split_ids.push(split_id.to_string()); + if !deletable_split_states.contains(&metadata.split_state) { + non_deletable_split_ids.push(split_id_ref.to_string()); continue; }; if metadata.split_state == SplitState::MarkedForDeletion { // If the split is already marked for deletion, This is fine, we just skip it. continue; } - metadata.split_state = SplitState::MarkedForDeletion; metadata.update_timestamp = now_timestamp; mutation_occurred = true; @@ -265,14 +265,20 @@ impl FileBackedIndex { /// Helper to mark a list of splits as published. /// This function however does not update the checkpoint. - fn mark_splits_as_published_helper(&mut self, split_ids: &[&str]) -> MetastoreResult<()> { + fn mark_splits_as_published_helper( + &mut self, + staged_split_ids: impl IntoIterator>, + ) -> MetastoreResult<()> { let mut split_not_found_ids = Vec::new(); let mut split_not_staged_ids = Vec::new(); + let now_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - for &split_id in split_ids { + + for staged_plit_id in staged_split_ids { + let staged_split_id_ref = staged_plit_id.as_ref(); // Check for the existence of split. - let Some(metadata) = self.splits.get_mut(split_id) else { - split_not_found_ids.push(split_id.to_string()); + let Some(metadata) = self.splits.get_mut(staged_split_id_ref) else { + split_not_found_ids.push(staged_split_id_ref.to_string()); continue; }; if metadata.split_state == SplitState::Staged { @@ -280,36 +286,35 @@ impl FileBackedIndex { metadata.update_timestamp = now_timestamp; metadata.publish_timestamp = Some(now_timestamp); } else { - split_not_staged_ids.push(split_id.to_string()); + split_not_staged_ids.push(staged_split_id_ref.to_string()); } } - if !split_not_found_ids.is_empty() { return Err(MetastoreError::SplitsDoNotExist { split_ids: split_not_found_ids, }); } - if !split_not_staged_ids.is_empty() { return Err(MetastoreError::SplitsNotStaged { split_ids: split_not_staged_ids, }); } - Ok(()) } /// Publishes splits. - pub(crate) fn publish_splits<'a>( + pub(crate) fn publish_splits( &mut self, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, + staged_split_ids: impl IntoIterator>, + replaced_split_ids: impl IntoIterator>, + checkpoint_delta_opt: Option, ) -> MetastoreResult<()> { if let Some(checkpoint_delta) = checkpoint_delta_opt { - self.metadata.checkpoint.try_apply_delta(checkpoint_delta)?; + self.metadata + .checkpoint + .try_apply_delta(checkpoint_delta.into())?; } - self.mark_splits_as_published_helper(split_ids)?; + self.mark_splits_as_published_helper(staged_split_ids)?; self.mark_splits_for_deletion(replaced_split_ids, &[SplitState::Published], true)?; Ok(()) } @@ -319,15 +324,34 @@ impl FileBackedIndex { let limit = query.limit.unwrap_or(usize::MAX); let offset = query.offset.unwrap_or_default(); - let splits: Vec = self - .splits - .values() - .filter(|split| split_query_predicate(split, &query)) - .skip(offset) - .take(limit) - .cloned() - .collect(); - + let splits: Vec = if query.sort_by_staleness { + self.splits + .values() + .filter(|split| split_query_predicate(split, &query)) + .sorted_unstable_by(|left_split, right_split| { + left_split + .split_metadata + .delete_opstamp + .cmp(&right_split.split_metadata.delete_opstamp) + .then_with(|| { + left_split + .publish_timestamp + .cmp(&right_split.publish_timestamp) + }) + }) + .skip(offset) + .take(limit) + .cloned() + .collect() + } else { + self.splits + .values() + .filter(|split| split_query_predicate(split, &query)) + .skip(offset) + .take(limit) + .cloned() + .collect() + }; Ok(splits) } @@ -344,18 +368,26 @@ impl FileBackedIndex { } /// Deletes multiple splits. - pub(crate) fn delete_splits(&mut self, split_ids: &[&str]) -> MetastoreResult<()> { + pub(crate) fn delete_splits( + &mut self, + split_ids: impl IntoIterator>, + ) -> MetastoreResult<()> { + let mut num_deleted_splits = 0; let mut split_not_found_ids = Vec::new(); let mut split_not_deletable_ids = Vec::new(); - for &split_id in split_ids { - match self.delete_split(split_id) { - DeleteSplitOutcome::Success => {} + for split_id in split_ids { + let split_id_ref = split_id.as_ref(); + + match self.delete_split(split_id_ref) { + DeleteSplitOutcome::Success => { + num_deleted_splits += 1; + } DeleteSplitOutcome::SplitNotFound => { - split_not_found_ids.push(split_id); + split_not_found_ids.push(split_id_ref.to_string()); } DeleteSplitOutcome::Forbidden => { - split_not_deletable_ids.push(split_id.to_string()); + split_not_deletable_ids.push(split_id_ref.to_string()); } } } @@ -364,7 +396,7 @@ impl FileBackedIndex { split_ids: split_not_deletable_ids, }); } - info!(index_id=%self.index_id(), "Deleted {} splits from index.", split_ids.len()); + info!(index_id=%self.index_id(), "Deleted {num_deleted_splits} splits from index."); if !split_not_found_ids.is_empty() { warn!( @@ -378,8 +410,8 @@ impl FileBackedIndex { } /// Adds a source. - pub(crate) fn add_source(&mut self, source: SourceConfig) -> MetastoreResult<()> { - self.metadata.add_source(source) + pub(crate) fn add_source(&mut self, source_config: SourceConfig) -> MetastoreResult<()> { + self.metadata.add_source(source_config) } pub(crate) fn toggle_source(&mut self, source_id: &str, enable: bool) -> MetastoreResult { @@ -423,16 +455,16 @@ impl FileBackedIndex { /// Updates splits delete opstamp. Returns that a mutation occurred (true). pub(crate) fn update_splits_delete_opstamp( &mut self, - split_ids: &[&str], + split_ids: impl IntoIterator>, delete_opstamp: u64, ) -> MetastoreResult { for split_id in split_ids { - let split = - self.splits - .get_mut(*split_id) - .ok_or_else(|| MetastoreError::SplitsDoNotExist { - split_ids: vec![split_id.to_string()], - })?; + let split_id_ref = split_id.as_ref(); + let split = self.splits.get_mut(split_id_ref).ok_or_else(|| { + MetastoreError::SplitsDoNotExist { + split_ids: vec![split_id_ref.to_string()], + } + })?; split.split_metadata.delete_opstamp = delete_opstamp; } Ok(true) @@ -475,32 +507,27 @@ impl Debug for Stamper { } fn split_query_predicate(split: &&Split, query: &ListSplitsQuery) -> bool { - if !split_tag_filter(split, query.tags.as_ref()) { + if !split_tag_filter(&split.split_metadata, query.tags.as_ref()) { return false; } - if !query.split_states.is_empty() && !query.split_states.contains(&split.split_state) { return false; } - if !query .delete_opstamp .contains(&split.split_metadata.delete_opstamp) { return false; } - if !query.update_timestamp.contains(&split.update_timestamp) { return false; } - if !query .create_timestamp .contains(&split.split_metadata.create_timestamp) { return false; } - match &query.mature { Bound::Included(evaluation_datetime) => { return split.split_metadata.is_mature(*evaluation_datetime); @@ -510,13 +537,11 @@ fn split_query_predicate(split: &&Split, query: &ListSplitsQuery) -> bool { } Bound::Unbounded => {} } - - if let Some(range) = split.split_metadata.time_range.as_ref() { + if let Some(range) = &split.split_metadata.time_range { if !query.time_range.overlaps_with(range.clone()) { return false; } } - true } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs index fd8e60730ea..30fdf05b196 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -33,8 +33,14 @@ use std::time::Duration; use async_trait::async_trait; use futures::future::try_join_all; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use quickwit_storage::Storage; use tokio::sync::{Mutex, OwnedMutexGuard, RwLock}; @@ -46,11 +52,11 @@ use self::store_operations::{ check_indexes_states_exist, delete_index, fetch_index, fetch_or_init_indexes_states, index_exists, put_index, put_indexes_states, }; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{ - IndexMetadata, ListSplitsQuery, Metastore, MetastoreError, MetastoreResult, Split, - SplitMetadata, SplitState, +use super::{ + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, ListIndexesResponseExt, + ListSplitsRequestExt, ListSplitsResponseExt, StageSplitsRequestExt, }; +use crate::{IndexMetadata, Metastore, MetastoreError, MetastoreResult, SplitState}; /// State of an index tracked by the metastore. pub(crate) enum IndexState { @@ -307,8 +313,11 @@ impl FileBackedMetastore { impl Metastore for FileBackedMetastore { /// ------------------------------------------------------------------------------- /// Mutations over the high-level index. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_id = index_config.index_id.clone(); + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_id = request.index_id.clone(); // We pick the outer lock here, so that we enter a critical section. let mut per_index_metastores_wlock = self.per_index_metastores.write().await; @@ -343,6 +352,7 @@ impl Metastore for FileBackedMetastore { } // Put index metadata on storage. + let index_config = request.deserialize_index_config()?; let index_metadata = IndexMetadata::new(index_config); let index_uid = index_metadata.index_uid.clone(); let index = FileBackedIndex::from(index_metadata); @@ -364,13 +374,17 @@ impl Metastore for FileBackedMetastore { per_index_metastores_wlock.insert(index_id.clone(), IndexState::Creating); } put_res?; - Ok(index_uid) + let response = CreateIndexResponse { + index_uid: index_uid.into(), + }; + Ok(response) } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { // We pick the outer lock here, so that we enter a critical section. let mut per_index_metastores_wlock = self.per_index_metastores.write().await; + let index_uid: IndexUid = request.index_uid.into(); let index_id = index_uid.index_id(); // If index is neither in `per_index_metastores_wlock` nor on the storage, it does not // exist. @@ -411,21 +425,20 @@ impl Metastore for FileBackedMetastore { }, _ => {} } - - delete_res + Ok(EmptyResponse {}) } /// ------------------------------------------------------------------------------- /// Mutations over a single index - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + let splits_metadata = request.deserialize_splits_metadata()?; + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { let mut failed_split_ids = Vec::new(); - for split_metadata in split_metadata_list { + + for split_metadata in splits_metadata { match index.stage_split(split_metadata) { Ok(()) => {} Err(MetastoreError::SplitsNotStaged { split_ids }) => { @@ -443,33 +456,37 @@ impl Metastore for FileBackedMetastore { } }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn publish_splits<'a>( + async fn publish_splits( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { + request: PublishSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.publish_splits(split_ids, replaced_split_ids, checkpoint_delta_opt)?; + index.publish_splits( + request.staged_split_ids, + request.replaced_split_ids, + request.checkpoint_delta, + )?; Ok(MutationOccurred::Yes(())) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index .mark_splits_for_deletion( - split_ids, + request.split_ids, &[ SplitState::Staged, SplitState::Published, @@ -480,83 +497,106 @@ impl Metastore for FileBackedMetastore { .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.delete_splits(split_ids)?; - Ok(MutationOccurred::Yes(())) + index.delete_splits(request.split_ids)?; + Ok(MutationOccurred::Yes(EmptyResponse {})) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.add_source(source)?; + index.add_source(source_config)?; Ok(MutationOccurred::Yes(())) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index - .toggle_source(source_id, enable) + .toggle_source(&request.source_id, request.enable) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.delete_source(source_id).map(MutationOccurred::from) + index + .delete_source(&request.source_id) + .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index - .reset_source_checkpoint(source_id) + .reset_source_checkpoint(&request.source_id) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } /// ------------------------------------------------------------------------------- /// Read-only accessors - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - let query_clone = query.clone(); - self.read(query.index_uid, |index| index.list_splits(query_clone)) - .await + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { + let list_splits_query = request.deserialize_list_splits_query()?; + let index_uid = request.index_uid.into(); + + let splits = self + .read(index_uid, |index| index.list_splits(list_splits_query)) + .await?; + let response = ListSplitsResponse::try_from_splits_metadata( + splits.into_iter().map(|split| split.split_metadata), + )?; + Ok(response) } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - self.read_any(index_id, |index| Ok(index.metadata().clone())) - .await + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let index_metadata = if let Some(index_uid) = request.index_uid() { + self.read(index_uid, |index| Ok(index.metadata().clone())) + .await + } else { + self.read_any(&request.index_id, |index| Ok(index.metadata().clone())) + .await + }?; + let response = IndexMetadataResponse::try_from_index_metadata(index_metadata)?; + Ok(response) } - async fn list_indexes_metadatas(&self) -> MetastoreResult> { + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { // Done in two steps: // 1) Get index IDs and release the lock on `per_index_metastores`. // 2) Get each index metadata. Note that each get will take a read lock on @@ -573,11 +613,12 @@ impl Metastore for FileBackedMetastore { .cloned() .collect() }; - let indexes_metadatas: Vec = - try_join_all(index_ids.iter().map(|index_id| async move { - match self.index_metadata(index_id).await { - Ok(index_metadata) => Ok(Some(index_metadata)), - Err(MetastoreError::IndexDoesNotExist { index_id: _ }) => Ok(None), + let indexes_metadata: Vec = + try_join_all(index_ids.into_iter().map(|index_id| async move { + let request = IndexMetadataRequest::new(index_id); + match self.index_metadata(request).await { + Ok(response) => Ok(Some(response.deserialize_index_metadata())), + Err(MetastoreError::IndexDoesNotExist { .. }) => Ok(None), Err(MetastoreError::InternalError { message, cause }) => { // Indexes can be in a transition state `Creating` or `Deleting`. // This is fine to ignore them. @@ -593,8 +634,9 @@ impl Metastore for FileBackedMetastore { .await? .into_iter() .flatten() - .collect(); - Ok(indexes_metadatas) + .collect::>()?; + let response = ListIndexesResponse::try_from_indexes_metadata(indexes_metadata)?; + Ok(response) } fn uri(&self) -> &Uri { @@ -625,33 +667,34 @@ impl Metastore for FileBackedMetastore { Ok(delete_task) } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index - .update_splits_delete_opstamp(split_ids, delete_opstamp) + .update_splits_delete_opstamp(request.split_ids, request.delete_opstamp) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + let delete_tasks = self - .read( - index_uid, - |index| Ok(index.list_delete_tasks(opstamp_start)), - ) + .read(index_uid, |index| { + Ok(index.list_delete_tasks(request.opstamp_start)) + }) .await??; - Ok(delete_tasks) + let response = ListDeleteTasksResponse { delete_tasks }; + Ok(response) } } @@ -768,8 +811,8 @@ mod tests { ); // Check index is returned by list indexes. - let indexes = metastore.list_indexes_metadatas().await.unwrap(); - assert_eq!(indexes.len(), 1); + let response = metastore.list_indexes(ListIndexesRequest {}).await.unwrap(); + assert_eq!(response.indexes_metadata_json.len(), 1); // Open a non-existent index. let metastore_error = metastore @@ -842,16 +885,15 @@ mod tests { let index_uid = metastore.create_index(index_config).await.unwrap(); // stage split - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await + let stage_request = StageSplitsRequestBuilder::for_index(index_uid.clone()) + .with_split_metadata(split_metadata) + .try_build() .unwrap(); + metastore.stage_splits(stage_request).await.unwrap(); // publish split fails - let err = metastore - .publish_splits(index_uid.clone(), &[split_id], &[], None) - .await; - assert!(err.is_err()); + let publish_request = PublishSplitsRequest::new(index_uid, [split_id], [], None); + metastore.publish_splits(publish_request).await.unwrap_err(); // empty let query = @@ -930,9 +972,11 @@ mod tests { .list_all_splits(index_uid.clone()) .await? .is_empty()); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await?; + let stage_request = StageSplitsRequestBuilder::for_index(index_uid.clone()) + .with_split_metadata(split_metadata) + .try_build() + .unwrap(); + metastore.stage_splits(stage_request).await?; assert_eq!(metastore.list_all_splits(index_uid.clone()).await?.len(), 1); Ok(()) } @@ -969,9 +1013,11 @@ mod tests { .list_all_splits(index_uid.clone()) .await? .is_empty()); - metastore_wrt - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await?; + let stage_request = StageSplitsRequestBuilder::for_index(index_uid.clone()) + .with_split_metadata(split_metadata) + .try_build() + .unwrap(); + metastore_wrt.stage_splits(stage_request).await?; assert!(metastore_read .list_all_splits(index_uid.clone()) .await? @@ -1019,19 +1065,19 @@ mod tests { ..Default::default() }; // stage split - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await + let stage_request = StageSplitsRequestBuilder::for_index(index_uid.clone()) + .with_split_metadata(split_metadata) + .try_build() .unwrap(); + metastore.stage_splits(stage_request).await.unwrap(); tokio::time::sleep(sleep_duration).await; // publish split let split_id = format!("split-{i}"); - metastore - .publish_splits(index_uid.clone(), &[&split_id], &[], None) - .await - .unwrap(); + let publish_request = + PublishSplitsRequest::new(index_uid, [split_id], [], None); + metastore.publish_splits(publish_request).await.unwrap(); } }); handles.push(handle); @@ -1061,17 +1107,18 @@ mod tests { // Delete indexes + call to list_indexes_metadata. let mut handles = Vec::new(); for index_uid in index_uids { + let delete_request = DeleteIndexRequest::new(index_uid); { let metastore = metastore.clone(); let handle = tokio::spawn(async move { - metastore.list_indexes_metadatas().await.unwrap(); + metastore.list_indexes(ListIndexesRequest {}).await.unwrap(); }); handles.push(handle); } { let metastore = metastore.clone(); let handle = tokio::spawn(async move { - metastore.delete_index(index_uid).await.unwrap(); + metastore.delete_index(delete_request).await.unwrap(); }); handles.push(handle); } @@ -1179,7 +1226,8 @@ mod tests { IndexState::Creating )); // Let's delete the index to clean states. - let deleted_index_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest::new(index_uid); + let deleted_index_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!( deleted_index_error, MetastoreError::IndexDoesNotExist { .. } @@ -1276,7 +1324,8 @@ mod tests { let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index - let metastore_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest::new(index_uid); + let metastore_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!( metastore_error, MetastoreError::InternalError { .. } @@ -1328,7 +1377,8 @@ mod tests { let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index - let metastore_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest::new(index_uid); + let metastore_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!( metastore_error, MetastoreError::InternalError { .. } @@ -1392,8 +1442,8 @@ mod tests { let metastore = FileBackedMetastore::try_new(ram_storage.clone(), None) .await .unwrap(); - let indexes_metadatas = metastore.list_indexes_metadatas().await.unwrap(); - assert_eq!(indexes_metadatas.len(), 1); + let list_indexes_response = metastore.list_indexes(ListIndexesRequest {}).await.unwrap(); + assert_eq!(list_indexes_response.num_indexes(), 1); // Fetch the index metadata not registered in indexes states json. metastore @@ -1403,17 +1453,17 @@ mod tests { // Now list indexes return 2 indexes metadatas as the metastore is now aware of // 2 alive indexes. - let indexes_metadatas = metastore.list_indexes_metadatas().await.unwrap(); - assert_eq!(indexes_metadatas.len(), 2); + let list_indexes_response = metastore.list_indexes(ListIndexesRequest {}).await.unwrap(); + assert_eq!(list_indexes_response.num_indexes(), 2); // Let's delete indexes. - metastore.delete_index(index_uid_alive).await.unwrap(); - metastore - .delete_index(index_uid_unregistered) - .await - .unwrap(); - let no_more_indexes = metastore.list_indexes_metadatas().await.unwrap(); - assert!(no_more_indexes.is_empty()); + let delete_request = DeleteIndexRequest::new(index_uid_alive); + metastore.delete_index(delete_request).await.unwrap(); + + let delete_request = DeleteIndexRequest::new(index_uid_unregistered); + metastore.delete_index(delete_request).await.unwrap(); + let list_indexes_response = metastore.list_indexes(ListIndexesRequest {}).await.unwrap(); + assert_eq!(list_indexes_response.num_indexes(), 2); Ok(()) } diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs index da1c4c10427..16a926c3ed9 100644 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs +++ b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs @@ -20,25 +20,20 @@ use std::sync::Arc; use async_trait::async_trait; -use itertools::Itertools; -use quickwit_config::IndexConfig; -use quickwit_proto::metastore::metastore_service_server::{self as grpc}; use quickwit_proto::metastore::{ - AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, - DeleteIndexResponse, DeleteQuery, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, - IndexMetadataRequest, IndexMetadataResponse, LastDeleteOpstampRequest, - LastDeleteOpstampResponse, ListAllSplitsRequest, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexesMetadatasRequest, ListIndexesMetadatasResponse, - ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, SourceResponse, SplitResponse, - StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, - UpdateSplitsDeleteOpstampResponse, + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, + ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, ListIndexesResponse, + ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, MetastoreService, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, }; +use quickwit_proto::set_parent_span_from_request_metadata; use quickwit_proto::tonic::{Request, Response, Status}; -use quickwit_proto::{set_parent_span_from_request_metadata, tonic}; use tracing::instrument; -use crate::{ListSplitsQuery, Metastore, MetastoreError}; +use crate::{Metastore, MetastoreError}; #[allow(missing_docs)] #[derive(Clone)] @@ -51,396 +46,226 @@ impl From> for GrpcMetastoreAdapter { } #[async_trait] -impl grpc::MetastoreService for GrpcMetastoreAdapter { +impl MetastoreService for GrpcMetastoreAdapter { #[instrument(skip(self, request))] async fn create_index( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let create_index_request = request.into_inner(); - let index_config = - serde_json::from_str::(&create_index_request.index_config_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "IndexConfig".to_string(), - message: error.to_string(), - })?; - let create_index_reply = - self.0 - .create_index(index_config) - .await - .map(|index_uid| CreateIndexResponse { - index_uid: index_uid.to_string(), - })?; - Ok(tonic::Response::new(create_index_reply)) + self.0 + .create_index(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn index_metadata( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let index_metadata_request = request.into_inner(); - let index_metadata = self - .0 - .index_metadata(&index_metadata_request.index_id) - .await?; - let index_metadata_reply = serde_json::to_string(&index_metadata) - .map(|index_metadata_serialized_json| IndexMetadataResponse { - index_metadata_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "IndexMetadata".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(index_metadata_reply)) + self.0 + .index_metadata(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] - async fn list_indexes_metadatas( + async fn list_indexes( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let indexes_metadatas = self.0.list_indexes_metadatas().await?; - let list_indexes_metadatas_reply = serde_json::to_string(&indexes_metadatas) - .map( - |indexes_metadatas_serialized_json| ListIndexesMetadatasResponse { - indexes_metadatas_serialized_json, - }, - ) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_indexes_metadatas_reply)) + self.0 + .list_indexes(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn delete_index( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let delete_request = request.into_inner(); - let delete_reply = self - .0 - .delete_index(delete_request.index_uid.into()) + self.0 + .delete_index(request.into_inner()) .await - .map(|_| DeleteIndexResponse {})?; - Ok(tonic::Response::new(delete_reply)) - } - - #[instrument(skip(self, request))] - async fn list_all_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let list_all_splits_request = request.into_inner(); - let splits = self - .0 - .list_all_splits(list_all_splits_request.index_uid.into()) - .await?; - let list_all_splits_reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_all_splits_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn list_splits( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let list_splits_request = request.into_inner(); - let query: ListSplitsQuery = serde_json::from_str(&list_splits_request.filter_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "ListSplitsQuery".to_string(), - message: error.to_string(), - })?; - - let splits = self.0.list_splits(query).await?; - let list_splits_reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_splits_reply)) + self.0 + .list_splits(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn stage_splits( &self, request: Request, - ) -> Result, Status> { + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let stage_split_request = request.into_inner(); - let split_metadata_list = - serde_json::from_str(&stage_split_request.split_metadata_list_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; self.0 - .stage_splits(stage_split_request.index_uid.into(), split_metadata_list) - .await?; - Ok(tonic::Response::new(SplitResponse {})) + .stage_splits(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn publish_splits( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let publish_request = request.into_inner(); - let split_ids = publish_request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let replaced_split_ids = publish_request - .replaced_split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let checkpoint_delta_opt = publish_request - .index_checkpoint_delta_serialized_json - .map(|json| serde_json::from_str(&json)) - .transpose() - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "IndexCheckpointDelta".to_string(), - message: error.to_string(), - })?; - let publish_splits_reply = self - .0 - .publish_splits( - publish_request.index_uid.into(), - &split_ids, - &replaced_split_ids, - checkpoint_delta_opt, - ) + self.0 + .publish_splits(request.into_inner()) .await - .map(|_| SplitResponse {})?; - Ok(tonic::Response::new(publish_splits_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn mark_splits_for_deletion( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let mark_splits_for_deletion_request = request.into_inner(); - let split_ids = mark_splits_for_deletion_request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let mark_splits_for_deletion_reply = self - .0 - .mark_splits_for_deletion( - mark_splits_for_deletion_request.index_uid.into(), - &split_ids, - ) + self.0 + .mark_splits_for_deletion(request.into_inner()) .await - .map(|_| SplitResponse {})?; - Ok(tonic::Response::new(mark_splits_for_deletion_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn delete_splits( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let delete_splits_request = request.into_inner(); - let split_ids = delete_splits_request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let delete_splits_reply = self - .0 - .delete_splits(delete_splits_request.index_uid.into(), &split_ids) + self.0 + .delete_splits(request.into_inner()) .await - .map(|_| SplitResponse {})?; - Ok(tonic::Response::new(delete_splits_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn add_source( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let add_source_request = request.into_inner(); - let source_config = serde_json::from_str(&add_source_request.source_config_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "SourceConfig".to_string(), - message: error.to_string(), - })?; - let add_source_reply = self - .0 - .add_source(add_source_request.index_uid.into(), source_config) + self.0 + .add_source(request.into_inner()) .await - .map(|_| SourceResponse {})?; - Ok(tonic::Response::new(add_source_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn toggle_source( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let toggle_source_request = request.into_inner(); - let toggle_source_reply = self - .0 - .toggle_source( - toggle_source_request.index_uid.into(), - &toggle_source_request.source_id, - toggle_source_request.enable, - ) + self.0 + .toggle_source(request.into_inner()) .await - .map(|_| SourceResponse {})?; - Ok(tonic::Response::new(toggle_source_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn delete_source( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let delete_source_request = request.into_inner(); - let delete_source_reply = self - .0 - .delete_source( - delete_source_request.index_uid.into(), - &delete_source_request.source_id, - ) + self.0 + .delete_source(request.into_inner()) .await - .map(|_| SourceResponse {})?; - Ok(tonic::Response::new(delete_source_reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn reset_source_checkpoint( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let reply = self - .0 - .reset_source_checkpoint(request.index_uid.into(), &request.source_id) + self.0 + .reset_source_checkpoint(request.into_inner()) .await - .map(|_| SourceResponse {})?; - Ok(tonic::Response::new(reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn last_delete_opstamp( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); let request = request.into_inner(); let last_delete_opstamp = self.0.last_delete_opstamp(request.index_uid.into()).await?; - let last_opstamp_reply = LastDeleteOpstampResponse { + let response = LastDeleteOpstampResponse { last_delete_opstamp, }; - Ok(tonic::Response::new(last_opstamp_reply)) + Ok(Response::new(response)) } #[instrument(skip(self, request))] async fn create_delete_task( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let delete_task = self.0.create_delete_task(request).await?; - Ok(tonic::Response::new(delete_task)) + self.0 + .create_delete_task(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn update_splits_delete_opstamp( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let split_ids = request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let reply = self - .0 - .update_splits_delete_opstamp( - request.index_uid.into(), - &split_ids, - request.delete_opstamp, - ) + self.0 + .update_splits_delete_opstamp(request.into_inner()) .await - .map(|_| UpdateSplitsDeleteOpstampResponse {})?; - Ok(tonic::Response::new(reply)) + .map(Response::new) + .map_err(|error| error.into()) } #[instrument(skip(self, request))] async fn list_delete_tasks( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: Request, + ) -> Result, Status> { set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let delete_tasks = self - .0 - .list_delete_tasks(request.index_uid.into(), request.opstamp_start) - .await? - .into_iter() - .map(DeleteTask::from) - .collect_vec(); - let reply = ListDeleteTasksResponse { delete_tasks }; - Ok(tonic::Response::new(reply)) - } - - #[instrument(skip(self, request))] - async fn list_stale_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let splits = self - .0 - .list_stale_splits( - request.index_uid.into(), - request.delete_opstamp, - request.num_splits as usize, - ) - .await?; - let reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(reply)) + self.0 + .list_delete_tasks(request.into_inner()) + .await + .map(Response::new) + .map_err(|error| error.into()) } } diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs index 59b0c9a573a..f6799ceacc0 100644 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs @@ -25,16 +25,14 @@ use std::net::SocketAddr; use anyhow::bail; use async_trait::async_trait; pub use grpc_adapter::GrpcMetastoreAdapter; -use itertools::Itertools; use quickwit_common::tower::BalanceChannel; use quickwit_common::uri::Uri as QuickwitUri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::metastore_service_client::MetastoreServiceClient; use quickwit_proto::metastore::{ - AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, DeleteQuery, DeleteSourceRequest, - DeleteSplitsRequest, DeleteTask, IndexMetadataRequest, LastDeleteOpstampRequest, - ListAllSplitsRequest, ListDeleteTasksRequest, ListIndexesMetadatasRequest, ListSplitsRequest, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, PublishSplitsRequest, + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, LastDeleteOpstampRequest, ListDeleteTasksRequest, + ListDeleteTasksResponse, ListIndexesRequest, ListIndexesResponse, ListSplitsRequest, + ListSplitsResponse, MarkSplitsForDeletionRequest, MetastoreServiceClient, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, }; @@ -43,11 +41,7 @@ use quickwit_proto::tonic::Status; use quickwit_proto::{IndexUid, SpanContextInterceptor}; use tower::timeout::error::Elapsed; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{ - IndexMetadata, ListSplitsQuery, Metastore, MetastoreError, MetastoreResult, Split, - SplitMetadata, -}; +use crate::{Metastore, MetastoreError, MetastoreResult}; // URI describing in a generic way the metastore services resource present in the cluster (= // discovered by Quickwit gossip). This value is used to build the URI of `MetastoreGrpcClient` and @@ -136,305 +130,151 @@ impl Metastore for MetastoreGrpcClient { } /// Creates an index. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_config_serialized_json = - serde_json::to_string(&index_config).map_err(|error| { - MetastoreError::JsonSerializeError { - struct_name: "IndexConfig".to_string(), - message: error.to_string(), - } - })?; - let request = CreateIndexRequest { - index_config_serialized_json, - }; - let inner_response = self - .underlying + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + self.underlying .clone() .create_index(request) .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))? - .into_inner(); - let index_uid = inner_response.index_uid.into(); - Ok(index_uid) + .map(|tonic_response| tonic_response.into_inner()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// List indexes. - async fn list_indexes_metadatas(&self) -> MetastoreResult> { - let response = self - .underlying + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { + self.underlying .clone() - .list_indexes_metadatas(ListIndexesMetadatasRequest {}) + .list_indexes(request) .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let indexes_metadatas = - serde_json::from_str(&response.into_inner().indexes_metadatas_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(indexes_metadatas) + .map(|tonic_response| tonic_response.into_inner()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Returns the [`IndexMetadata`] for a given index. - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - let request = IndexMetadataRequest { - index_id: index_id.to_string(), - }; - let response = self - .underlying + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + self.underlying .clone() .index_metadata(request) .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let index_metadata = serde_json::from_str( - &response.into_inner().index_metadata_serialized_json, - ) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "IndexMetadata".to_string(), - message: error.to_string(), - })?; - Ok(index_metadata) + .map(|tonic_response| tonic_response.into_inner()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Deletes an index. - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - let request = DeleteIndexRequest { - index_uid: index_uid.to_string(), - }; + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { self.underlying .clone() .delete_index(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Stages several splits. - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - let split_metadata_list_serialized_json = serde_json::to_string(&split_metadata_list) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - let tonic_request = StageSplitsRequest { - index_uid: index_uid.to_string(), - split_metadata_list_serialized_json, - }; + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { self.underlying .clone() - .stage_splits(tonic_request) + .stage_splits(request) .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map(|tonic_response| tonic_response.into_inner()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Publishes a list of splits. - async fn publish_splits<'a>( + async fn publish_splits( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids.iter().map(|split| split.to_string()).collect(); - let replaced_split_ids_vec: Vec = replaced_split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let index_checkpoint_delta_serialized_json = checkpoint_delta_opt - .map(|checkpoint_delta| serde_json::to_string(&checkpoint_delta)) - .transpose() - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "IndexCheckpointDelta".to_string(), - message: error.to_string(), - })?; - let request = PublishSplitsRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - replaced_split_ids: replaced_split_ids_vec, - index_checkpoint_delta_serialized_json, - }; + request: PublishSplitsRequest, + ) -> MetastoreResult { self.underlying .clone() .publish_splits(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Lists the splits. - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - let filter_json = - serde_json::to_string(&query).map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "ListSplitsQuery".to_string(), - message: error.to_string(), - })?; - - let request = ListSplitsRequest { filter_json }; - let response = self - .underlying + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { + self.underlying .clone() .list_splits(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) - } - - /// Lists all the splits without filtering. - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - let request = ListAllSplitsRequest { - index_uid: index_uid.into(), - }; - let response = self - .underlying - .clone() - .list_all_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Marks a list of splits for deletion. - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = MarkSplitsForDeletionRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - }; + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { self.underlying .clone() .mark_splits_for_deletion(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Deletes a list of splits. - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = DeleteSplitsRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - }; + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { self.underlying .clone() .delete_splits(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Adds a source to a given index. - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - let source_config_serialized_json = - serde_json::to_string(&source).map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "SourceConfig".to_string(), - message: error.to_string(), - })?; - let request = AddSourceRequest { - index_uid: index_uid.into(), - source_config_serialized_json, - }; + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { self.underlying .clone() .add_source(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Toggles the source `enabled` field value. - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - let request = ToggleSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - enable, - }; + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { self.underlying .clone() .toggle_source(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Removes a source from a given index. - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - let request = DeleteSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - }; + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { self.underlying .clone() .delete_source(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } /// Resets a source checkpoint. async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - let request = ResetSourceCheckpointRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - }; + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { self.underlying .clone() .reset_source_checkpoint(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { @@ -462,80 +302,28 @@ impl Metastore for MetastoreGrpcClient { Ok(response) } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = UpdateSplitsDeleteOpstampRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - delete_opstamp, - }; + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { self.underlying .clone() .update_splits_delete_opstamp(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - let request = ListDeleteTasksRequest { - index_uid: index_uid.into(), - opstamp_start, - }; - let response = self - .underlying + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + self.underlying .clone() .list_delete_tasks(request) .await .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let delete_tasks: Vec = response - .delete_tasks - .into_iter() - .map(DeleteTask::from) - .collect_vec(); - Ok(delete_tasks) - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - let request = ListStaleSplitsRequest { - index_uid: index_uid.into(), - delete_opstamp, - num_splits: num_splits as u64, - }; - let response = self - .underlying - .clone() - .list_stale_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) + .map_err(|tonic_error| parse_grpc_error(&tonic_error)) } } diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index 489dc727033..990d57fcf67 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -24,10 +24,12 @@ use std::collections::{BTreeMap, HashMap}; use quickwit_common::uri::Uri; use quickwit_config::{IndexConfig, SourceConfig, TestableForRegression}; +use quickwit_proto::metastore::IndexMetadataResponse; use quickwit_proto::IndexUid; use serde::{Deserialize, Serialize}; use serialize::VersionedIndexMetadata; use time::OffsetDateTime; +use tracing::error; use crate::checkpoint::{ IndexCheckpoint, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, @@ -90,16 +92,17 @@ impl IndexMetadata { } /// Adds a source to the index. Returns an error if the source_id already exists. - pub fn add_source(&mut self, source: SourceConfig) -> MetastoreResult<()> { - let entry = self.sources.entry(source.source_id.clone()); - let source_id = source.source_id.clone(); + pub fn add_source(&mut self, source_config: SourceConfig) -> MetastoreResult<()> { + let entry = self.sources.entry(source_config.source_id.clone()); + let source_id = source_config.source_id.clone(); + if let Entry::Occupied(_) = entry { return Err(MetastoreError::SourceAlreadyExists { source_id: source_id.clone(), - source_type: source.source_type().to_string(), + source_type: source_config.source_type().to_string(), }); } - entry.or_insert(source); + entry.or_insert(source_config); self.checkpoint.add_source(&source_id); Ok(()) } @@ -128,6 +131,23 @@ impl IndexMetadata { } } +impl TryFrom for IndexMetadata { + type Error = MetastoreError; + + fn try_from(response: IndexMetadataResponse) -> Result { + let index_metadata = serde_json::from_str::(&response.index_metadata_json) + .map_err(|error| { + error!(error=?error, "Failed to deserialize index metadata."); + + MetastoreError::JsonDeserializeError { + struct_name: "IndexMetadata".to_string(), + message: error.to_string(), + } + })?; + Ok(index_metadata) + } +} + impl TestableForRegression for IndexMetadata { fn sample_for_regression() -> IndexMetadata { let mut source_checkpoint = SourceCheckpoint::default(); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs index 75a0a02846e..d4aeed4f39a 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; +use anyhow::ensure; use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_proto::IndexUid; use serde::{self, Deserialize, Serialize}; @@ -90,9 +91,11 @@ impl TryFrom for IndexMetadata { fn try_from(v0_6: IndexMetadataV0_6) -> anyhow::Result { let mut sources: HashMap = Default::default(); for source in v0_6.sources { - if sources.contains_key(&source.source_id) { - anyhow::bail!("Source `{}` is defined more than once", source.source_id); - } + ensure!( + !sources.contains_key(&source.source_id), + "Source `{}` is defined multiple times.", + source.source_id + ); sources.insert(source.source_id.clone(), source); } Ok(Self { diff --git a/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs b/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs index 5f366c176f6..12698bb371d 100644 --- a/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs @@ -19,12 +19,17 @@ use async_trait::async_trait; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListSplitsQuery, Metastore, MetastoreResult, Split, SplitMetadata}; +use crate::{Metastore, MetastoreResult}; macro_rules! instrument { ($expr:expr, [$operation:ident, $($label:expr),*]) => { @@ -84,163 +89,134 @@ impl Metastore for InstrumentedMetastore { } // Index API - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_id = index_config.index_id.clone(); - instrument!( - self.underlying.create_index(index_config).await, - [create_index, &index_id] - ); - } - async fn index_exists(&self, index_id: &str) -> MetastoreResult { + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_id = "FIXME"; instrument!( - self.underlying.index_exists(index_id).await, - [index_exists, index_id] + self.underlying.create_index(request).await, + [create_index, index_id] ); } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let index_id = request.index_id.clone(); instrument!( - self.underlying.index_metadata(index_id).await, - [index_metadata, index_id] + self.underlying.index_metadata(request).await, + [index_metadata, &index_id] ); } - async fn list_indexes_metadatas(&self) -> MetastoreResult> { + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { instrument!( - self.underlying.list_indexes_metadatas().await, + self.underlying.list_indexes(request).await, [list_indexes_metadatas, ""] ); } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying.delete_index(index_uid.clone()).await, + self.underlying.delete_index(request).await, [delete_index, index_uid.index_id()] ); } // Split API - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .stage_splits(index_uid.clone(), split_metadata_list) - .await, + self.underlying.stage_splits(request).await, [stage_splits, index_uid.index_id()] ); } - async fn publish_splits<'a>( + async fn publish_splits( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { + request: PublishSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .publish_splits( - index_uid.clone(), - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - ) - .await, + self.underlying.publish_splits(request).await, [publish_splits, index_uid.index_id()] ); } - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying.list_splits(query.clone()).await, - [list_splits, query.index_uid.index_id()] + self.underlying.list_splits(request).await, + [list_splits, index_uid.index_id()] ); } - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - instrument!( - self.underlying.list_all_splits(index_uid.clone()).await, - [list_all_splits, index_uid.index_id()] - ); - } - - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .mark_splits_for_deletion(index_uid.clone(), split_ids) - .await, + self.underlying.mark_splits_for_deletion(request).await, [mark_splits_for_deletion, index_uid.index_id()] ); } - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .delete_splits(index_uid.clone(), split_ids) - .await, + self.underlying.delete_splits(request).await, [delete_splits, index_uid.index_id()] ); } // Source API - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying.add_source(index_uid.clone(), source).await, + self.underlying.add_source(request).await, [add_source, index_uid.index_id()] ); } - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .toggle_source(index_uid.clone(), source_id, enable) - .await, + self.underlying.toggle_source(request).await, [toggle_source, index_uid.index_id()] ); } async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .reset_source_checkpoint(index_uid.clone(), source_id) - .await, + self.underlying.reset_source_checkpoint(request).await, [reset_source_checkpoint, index_uid.index_id()] ); } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .delete_source(index_uid.clone(), source_id) - .await, + self.underlying.delete_source(request).await, [delete_source, index_uid.index_id()] ); } // Delete task API + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { let index_uid: IndexUid = delete_query.index_uid.clone().into(); instrument!( @@ -251,13 +227,11 @@ impl Metastore for InstrumentedMetastore { async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .list_delete_tasks(index_uid.clone(), opstamp_start) - .await, + self.underlying.list_delete_tasks(request).await, [list_delete_tasks, index_uid.index_id()] ); } @@ -269,33 +243,16 @@ impl Metastore for InstrumentedMetastore { ); } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.clone().into(); instrument!( - self.underlying - .update_splits_delete_opstamp(index_uid.clone(), split_ids, delete_opstamp) - .await, + self.underlying.update_splits_delete_opstamp(request).await, [update_splits_delete_opstamp, index_uid.index_id()] ); } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - instrument!( - self.underlying - .list_stale_splits(index_uid.clone(), delete_opstamp, num_splits) - .await, - [list_stale_splits, index_uid.index_id()] - ); - } } #[cfg(test)] diff --git a/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs b/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs index c9cf8afe2eb..3850e7b3796 100644 --- a/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs +++ b/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs @@ -24,12 +24,19 @@ use async_trait::async_trait; use quickwit_common::pubsub::{Event, EventBroker}; use quickwit_common::uri::Uri; use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use tracing::info; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListSplitsQuery, Metastore, MetastoreResult, Split, SplitMetadata}; +use super::AddSourceRequestExt; +use crate::{ListSplitsQuery, Metastore, MetastoreResult, Split}; /// Metastore events dispatched to subscribers. #[derive(Debug, Clone, Eq, PartialEq)] @@ -101,135 +108,104 @@ impl Metastore for MetastoreEventPublisher { } // Index API - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - self.underlying.create_index(index_config).await - } - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - self.underlying.index_exists(index_id).await + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + self.underlying.create_index(request).await } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - self.underlying.index_metadata(index_id).await + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + self.underlying.index_metadata(request).await } - async fn list_indexes_metadatas(&self) -> MetastoreResult> { - self.underlying.list_indexes_metadatas().await + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { + self.underlying.list_indexes(request).await } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { let event = MetastoreEvent::DeleteIndex { - index_uid: index_uid.clone(), + index_uid: request.index_uid.clone().into(), }; - self.underlying.delete_index(index_uid).await?; + let response = self.underlying.delete_index(request).await?; self.event_broker.publish(event); - Ok(()) + Ok(response) } // Split API - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - self.underlying - .stage_splits(index_uid, split_metadata_list) - .await + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + self.underlying.stage_splits(request).await } - async fn publish_splits<'a>( + async fn publish_splits( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { - self.underlying - .publish_splits( - index_uid, - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - ) - .await + request: PublishSplitsRequest, + ) -> MetastoreResult { + self.underlying.publish_splits(request).await } - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - self.underlying.list_splits(query).await + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { + self.underlying.list_splits(request).await } - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - self.underlying.list_all_splits(index_uid).await - } - - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.underlying - .mark_splits_for_deletion(index_uid, split_ids) - .await + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + self.underlying.mark_splits_for_deletion(request).await } - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.underlying.delete_splits(index_uid, split_ids).await + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + self.underlying.delete_splits(request).await } // Source API - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; let event = MetastoreEvent::AddSource { - index_uid: index_uid.clone(), - source_config: source.clone(), + index_uid: request.index_uid.clone().into(), + source_config, }; - info!("add source {0}, {source:?}", index_uid.index_id()); - self.underlying.add_source(index_uid, source).await?; + let response = self.underlying.add_source(request).await?; self.event_broker.publish(event); - Ok(()) + Ok(response) } - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let event = MetastoreEvent::ToggleSource { - index_uid: index_uid.clone(), - source_id: source_id.to_string(), - enabled: enable, + index_uid: request.index_uid.clone().into(), + source_id: request.source_id.clone(), + enabled: request.enable, }; - self.underlying - .toggle_source(index_uid, source_id, enable) - .await?; + let response = self.underlying.toggle_source(request).await?; self.event_broker.publish(event); - Ok(()) + Ok(response) } async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - self.underlying - .reset_source_checkpoint(index_uid, source_id) - .await + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + self.underlying.reset_source_checkpoint(request).await } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let event = MetastoreEvent::DeleteSource { - index_uid: index_uid.clone(), - source_id: source_id.to_string(), + index_uid: request.index_uid.clone().into(), + source_id: request.source_id.clone(), }; - self.underlying.delete_source(index_uid, source_id).await?; + let response = self.underlying.delete_source(request).await?; self.event_broker.publish(event); - Ok(()) + Ok(response) } // Delete task API @@ -239,38 +215,20 @@ impl Metastore for MetastoreEventPublisher { async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - self.underlying - .list_delete_tasks(index_uid, opstamp_start) - .await + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + self.underlying.list_delete_tasks(request).await } async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { self.underlying.last_delete_opstamp(index_uid).await } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - self.underlying - .update_splits_delete_opstamp(index_uid, split_ids, delete_opstamp) - .await - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - self.underlying - .list_stale_splits(index_uid, delete_opstamp, num_splits) - .await + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + self.underlying.update_splits_delete_opstamp(request).await } } @@ -279,8 +237,10 @@ mod tests { use quickwit_common::pubsub::EventSubscriber; use quickwit_config::SourceParams; + use quickwit_proto::DeleteIndexRequest; use super::*; + use crate::metastore::CreateIndexRequestExt; use crate::metastore_for_test; use crate::tests::test_suite::DefaultForTest; @@ -313,32 +273,39 @@ mod tests { let (tx, mut rx) = tokio::sync::mpsc::channel(1); let subscription = metastore.event_broker.subscribe(TxSubscriber(tx)); - let index_uid = IndexUid::new("test-index"); + let index_id = "test-index"; + let index_uid = IndexUid::new(index_id); let index_uri = "ram:///indexes/test-index"; + let index_config = IndexConfig::for_test(index_id, index_uri); + + let create_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid = metastore.create_index(create_request).await.unwrap(); + let source_id = "test-source"; let source_config = SourceConfig::for_test(source_id, SourceParams::void()); + let add_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config).unwrap(); - let index_uid = metastore - .create_index(IndexConfig::for_test(index_uid.index_id(), index_uri)) - .await - .unwrap(); + metastore.add_source(add_request).await.unwrap(); + let toggle_request = ToggleSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.to_string(), + enable: false, + }; + metastore.toggle_source(toggle_request).await.unwrap(); + + let delete_source_request = DeleteSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.to_string(), + }; metastore - .add_source( - index_uid.clone(), - SourceConfig::for_test(source_id, SourceParams::void()), - ) - .await - .unwrap(); - metastore - .toggle_source(index_uid.clone(), source_id, false) - .await - .unwrap(); - metastore - .delete_source(index_uid.clone(), source_id) + .delete_source(delete_source_request) .await .unwrap(); - metastore.delete_index(index_uid.clone()).await.unwrap(); + + let delete_request = DeleteIndexRequest::new(index_uid.clone()); + metastore.delete_index(delete_request).await.unwrap(); assert_eq!( rx.recv().await.unwrap(), diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 773575d4724..55f92dc46c9 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -35,12 +35,18 @@ pub use index_metadata::IndexMetadata; use quickwit_common::uri::Uri; use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_doc_mapper::tag_pruning::TagFilterAst; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use time::OffsetDateTime; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{MetastoreError, MetastoreResult, Split, SplitMetadata, SplitState}; +use crate::{MetastoreError, MetastoreResult, SplitMetadata, SplitState}; /// Metastore meant to manage Quickwit's indexes, their splits and delete tasks. /// @@ -103,51 +109,32 @@ pub trait Metastore: Send + Sync + 'static { /// /// This API creates a new index in the metastore. /// An error will occur if an index that already exists in the storage is specified. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult; - - /// Returns whether the index `index_id` exists in the metastore. - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - match self.index_metadata(index_id).await { - Ok(_) => Ok(true), - Err(MetastoreError::IndexDoesNotExist { .. }) => Ok(false), - Err(error) => Err(error), - } - } - - /// Returns index uid for the given index. - async fn index_uid(&self, index_id: &str) -> MetastoreResult { - let index_uid = self.index_metadata(index_id).await?.index_uid; - Ok(index_uid) - } + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult; /// Returns the [`IndexMetadata`] of an index identified by its ID. - /// TODO consider merging with list_splits to remove one round-trip - async fn index_metadata(&self, index_id: &str) -> MetastoreResult; - - /// Returns the [`IndexMetadata`] of an index identified by its UID. - async fn index_metadata_strict(&self, index_uid: &IndexUid) -> MetastoreResult { - let index_metadata = self.index_metadata(index_uid.index_id()).await?; - - if index_metadata.index_uid != *index_uid { - return Err(MetastoreError::IndexDoesNotExist { - index_id: index_uid.index_id().to_string(), - }); - } - Ok(index_metadata) - } + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult; /// Lists the indexes. /// /// This API lists the indexes stored in the metastore and returns a collection of /// [`IndexMetadata`]. - async fn list_indexes_metadatas(&self) -> MetastoreResult>; + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult; /// Deletes an index. /// /// This API removes the specified from the metastore, but does not remove the index from the /// storage. An error will occur if an index that does not exist in the storage is /// specified. - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()>; + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult; // Split API @@ -160,11 +147,7 @@ pub trait Metastore: Send + Sync + 'static { /// /// A split needs to be staged before uploading any of its files to the storage. /// An error will occur if an index that does not exist in the storage is specified. - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()>; + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult; /// Publishes a set of staged splits while optionally marking another set of published splits /// for deletion. @@ -176,69 +159,25 @@ pub trait Metastore: Send + Sync + 'static { /// /// This method can be used to advance the checkpoint, by supplying an empty array for /// `staged_split_ids`. - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()>; + async fn publish_splits(&self, request: PublishSplitsRequest) + -> MetastoreResult; /// Lists the splits. /// /// Returns a list of splits that intersects the given `time_range`, `split_state`, and `tag`. /// Regardless of the time range filter, if a split has no timestamp it is always returned. /// An error will occur if an index that does not exist in the storage is specified. - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult>; - - /// Lists all the splits without filtering. - /// - /// Returns a list of all splits currently known to the metastore regardless of their state. - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - let query = ListSplitsQuery::for_index(index_uid); - self.list_splits(query).await - } - - /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_uid`. - /// These splits are called "stale" as they have an `delete_opstamp` strictly inferior - /// to the given `delete_opstamp`. - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - let query = ListSplitsQuery::for_index(index_uid) - .with_delete_opstamp_lt(delete_opstamp) - .with_split_state(SplitState::Published) - .retain_mature(OffsetDateTime::now_utc()); - - let mut splits = self.list_splits(query).await?; - splits.sort_by(|split_left, split_right| { - split_left - .split_metadata - .delete_opstamp - .cmp(&split_right.split_metadata.delete_opstamp) - .then_with(|| { - split_left - .publish_timestamp - .cmp(&split_right.publish_timestamp) - }) - }); - splits.truncate(num_splits); - Ok(splits) - } + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult; /// Marks a list of splits for deletion. /// /// This API will change the state to [`SplitState::MarkedForDeletion`] so that it is not /// referenced by the client anymore. It actually does not remove the split from storage. An /// error will occur if you specify an index or split that does not exist in the storage. - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()>; + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult; /// Deletes a list of splits. /// @@ -246,11 +185,7 @@ pub trait Metastore: Send + Sync + 'static { /// [`SplitState::MarkedForDeletion`] state. This removes the split metadata from the /// metastore, but does not remove the split from storage. An error will occur if you /// specify an index or split that does not exist in the storage. - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()>; + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult; // Source API @@ -259,23 +194,17 @@ pub trait Metastore: Send + Sync + 'static { /// same ID is already defined for the index. /// /// If a checkpoint is already registered for the source, it is kept. - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()>; + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult; /// Enables or Disables a source. /// Fails with `SourceDoesNotExist` error if the specified source doesn't exist. - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()>; + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult; /// Resets the checkpoint of a source identified by `index_uid` and `source_id`. async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()>; + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult; /// Deletes a source. Fails with /// [`SourceDoesNotExist`](crate::MetastoreError::SourceDoesNotExist) if the specified source @@ -283,7 +212,7 @@ pub trait Metastore: Send + Sync + 'static { /// /// The checkpoint associated to the source is deleted as well. /// If the checkpoint is missing, this does not trigger an error. - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()>; + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult; // Delete tasks API @@ -294,27 +223,295 @@ pub trait Metastore: Send + Sync + 'static { async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult; /// Updates splits `split_metadata.delete_opstamp` to the value `delete_opstamp`. - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()>; + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult; /// Lists [`DeleteTask`] with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult>; + request: ListDeleteTasksRequest, + ) -> MetastoreResult; +} + +pub trait CreateIndexRequestExt { + fn try_from_index_config(index_config: IndexConfig) -> MetastoreResult; + + fn deserialize_index_config(&self) -> MetastoreResult; +} + +impl CreateIndexRequestExt for CreateIndexRequest { + fn try_from_index_config(index_config: IndexConfig) -> MetastoreResult { + let index_config_json = serde_json::to_string(&index_config).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "IndexConfig".to_string(), + message: format!("Failed to serialize index config: {error:?}"), + } + })?; + let index_id = index_config.index_id; + let request = Self { + index_id, + index_config_json, + }; + Ok(request) + } + + fn deserialize_index_config(&self) -> MetastoreResult { + serde_json::from_str(&self.index_config_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "IndexConfig".to_string(), + message: format!("Failed to deserialize index config: {error:?}"), + } + }) + } +} + +pub trait IndexMetadataResponseExt { + fn try_from_index_metadata( + index_metadata: IndexMetadata, + ) -> MetastoreResult; + + fn deserialize_index_metadata(&self) -> MetastoreResult; +} + +impl IndexMetadataResponseExt for IndexMetadataResponse { + fn try_from_index_metadata(index_metadata: IndexMetadata) -> MetastoreResult { + let index_metadata_json = serde_json::to_string(&index_metadata).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "IndexMetadata".to_string(), + message: format!("Failed to serialize index metadata: {error:?}"), + } + })?; + let request = Self { + index_metadata_json, + }; + Ok(request) + } + + fn deserialize_index_metadata(&self) -> MetastoreResult { + serde_json::from_str(&self.index_metadata_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "IndexMetadata".to_string(), + message: format!("Failed to deserialize index metadata: {error:?}"), + } + }) + } +} + +pub trait ListIndexesResponseExt { + fn try_from_indexes_metadata( + indexes_metadata: impl IntoIterator, + ) -> MetastoreResult; + + fn deserialize_indexes_metadata(&self) -> MetastoreResult>; +} + +impl ListIndexesResponseExt for ListIndexesResponse { + fn try_from_indexes_metadata( + indexes_metadata: impl IntoIterator, + ) -> MetastoreResult { + let indexes_metadata: Vec = indexes_metadata.into_iter().collect(); + let indexes_metadata_json = serde_json::to_string(&indexes_metadata).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to serialize indexes metadata: {error:?}"), + } + })?; + let request = Self { + indexes_metadata_json, + }; + Ok(request) + } + + fn deserialize_indexes_metadata(&self) -> MetastoreResult> { + serde_json::from_str(&self.indexes_metadata_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to deserialize indexes metadata: {error:?}"), + } + }) + } +} + +pub trait AddSourceRequestExt { + fn try_from_source_config( + index_uid: impl Into, + source_config: SourceConfig, + ) -> MetastoreResult; + + fn deserialize_source_config(&self) -> MetastoreResult; +} + +impl AddSourceRequestExt for AddSourceRequest { + fn try_from_source_config( + index_uid: impl Into, + source_config: SourceConfig, + ) -> MetastoreResult { + let source_config_json = serde_json::to_string(&source_config).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "SourceConfig".to_string(), + message: format!("Failed to serialize source config: {error:?}"), + } + })?; + let request = Self { + index_uid: index_uid.into().into(), + source_config_json, + }; + Ok(request) + } + + fn deserialize_source_config(&self) -> MetastoreResult { + serde_json::from_str(&self.source_config_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "SourceConfig".to_string(), + message: format!("Failed to deserialize source config: {error:?}"), + } + }) + } +} + +pub trait StageSplitsRequestExt { + fn try_from_split_metadata( + index_uid: impl Into, + split_metadata: SplitMetadata, + ) -> MetastoreResult; + + fn try_from_splits_metadata( + index_uid: impl Into, + splits_metadata: impl IntoIterator, + ) -> MetastoreResult; + + fn deserialize_splits_metadata(&self) -> MetastoreResult>; +} + +impl StageSplitsRequestExt for StageSplitsRequest { + fn try_from_split_metadata( + index_uid: impl Into, + split_metadata: SplitMetadata, + ) -> MetastoreResult { + let splits_metadata_json = serde_json::to_string(&[split_metadata]).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "SplitMetadata".to_string(), + message: format!("Failed to serialize split metadata: {error:?}"), + } + })?; + let request = Self { + index_uid: index_uid.into().into(), + splits_metadata_json, + }; + Ok(request) + } + + fn try_from_splits_metadata( + index_uid: impl Into, + splits_metadata: impl IntoIterator, + ) -> MetastoreResult { + let splits_metadata: Vec = splits_metadata.into_iter().collect(); + let splits_metadata_json = serde_json::to_string(&splits_metadata).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to serialize splits metadata: {error:?}"), + } + })?; + let request = Self { + index_uid: index_uid.into().into(), + splits_metadata_json, + }; + Ok(request) + } + + fn deserialize_splits_metadata(&self) -> MetastoreResult> { + serde_json::from_str(&self.splits_metadata_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to deserialize splits metadata: {error:?}"), + } + }) + } +} + +pub trait ListSplitsRequestExt { + fn try_from_list_splits_query( + index_uid: impl Into, + list_splits_query: ListSplitsQuery, + ) -> MetastoreResult; + + fn deserialize_list_splits_query(&self) -> MetastoreResult; +} + +impl ListSplitsRequestExt for ListSplitsRequest { + fn try_from_list_splits_query( + index_uid: impl Into, + list_splits_query: ListSplitsQuery, + ) -> MetastoreResult { + let list_splits_query_json = + serde_json::to_string(&list_splits_query).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "ListSplitsQuery".to_string(), + message: format!("Failed to serialize list splits query: {error:?}"), + } + })?; + let request = Self { + index_uid: index_uid.into().into(), + list_splits_query_json: Some(list_splits_query_json), + }; + Ok(request) + } + + fn deserialize_list_splits_query(&self) -> MetastoreResult { + let list_splits_query = self + .list_splits_query_json + .as_ref() + .map(|list_splits_query_json| serde_json::from_str(&list_splits_query_json)) + .transpose() + .map_err(|error| MetastoreError::JsonDeserializeError { + struct_name: "ListSplitsQuery".to_string(), + message: format!("Failed to deserialize list splits query: {error:?}"), + })? + .unwrap_or_default(); + Ok(list_splits_query) + } +} + +pub trait ListSplitsResponseExt { + fn try_from_splits_metadata( + splits_metadata: impl IntoIterator, + ) -> MetastoreResult; + + fn deserialize_splits_metadata(&self) -> MetastoreResult>; +} + +impl ListSplitsResponseExt for ListSplitsResponse { + fn try_from_splits_metadata( + splits_metadata: impl IntoIterator, + ) -> MetastoreResult { + let splits_metadata: Vec = splits_metadata.into_iter().collect(); + let splits_metadata_json = serde_json::to_string(&splits_metadata).map_err(|error| { + MetastoreError::JsonSerializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to serialize splits metadata: {error:?}"), + } + })?; + let request = Self { + splits_metadata_json, + }; + Ok(request) + } + + fn deserialize_splits_metadata(&self) -> MetastoreResult> { + serde_json::from_str(&self.splits_metadata_json).map_err(|error| { + MetastoreError::JsonDeserializeError { + struct_name: "Vec".to_string(), + message: format!("Failed to deserialize splits metadata: {error:?}"), + } + }) + } } #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] /// A query builder for listing splits within the metastore. pub struct ListSplitsQuery { - /// The index to get splits from. - pub index_uid: IndexUid, - /// The maximum number of splits to retrieve. pub limit: Option, @@ -341,14 +538,15 @@ pub struct ListSplitsQuery { /// The datetime at which you include or exclude mature splits. pub mature: Bound, + + /// Sorts the splits by staleness, i.e. by delete opstamp and publish timestamp in ascending + /// order. + pub sort_by_staleness: bool, } -#[allow(unused_attributes)] -impl ListSplitsQuery { - /// Creates a new [ListSplitsQuery] for a specific index. - pub fn for_index(index_uid: IndexUid) -> Self { +impl Default for ListSplitsQuery { + fn default() -> Self { Self { - index_uid, limit: None, offset: None, split_states: Vec::new(), @@ -358,9 +556,13 @@ impl ListSplitsQuery { update_timestamp: Default::default(), create_timestamp: Default::default(), mature: Bound::Unbounded, + sort_by_staleness: false, } } +} +#[allow(unused_attributes)] +impl ListSplitsQuery { /// Sets the maximum number of splits to retrieve. pub fn with_limit(mut self, n: usize) -> Self { self.limit = Some(n); @@ -514,6 +716,13 @@ impl ListSplitsQuery { self.mature = Bound::Excluded(now); self } + + /// Sorts the splits by staleness, i.e. by delete opstamp and publish timestamp in ascending + /// order. + pub fn sort_by_staleness(mut self) -> Self { + self.sort_by_staleness = true; + self + } } #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index 7a6ae58379d..1ca70415fab 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -28,11 +28,16 @@ use async_trait::async_trait; use itertools::Itertools; use quickwit_common::uri::Uri; use quickwit_common::PrettySample; -use quickwit_config::{ - IndexConfig, MetastoreBackend, MetastoreConfig, PostgresMetastoreConfig, SourceConfig, -}; +use quickwit_config::{MetastoreBackend, MetastoreConfig, PostgresMetastoreConfig}; use quickwit_doc_mapper::tag_pruning::TagFilterAst; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use sqlx::migrate::Migrator; use sqlx::postgres::{PgConnectOptions, PgDatabaseError, PgPoolOptions}; @@ -41,12 +46,11 @@ use tokio::sync::Mutex; use tracing::log::LevelFilter; use tracing::{debug, error, info, instrument, warn}; +use super::{CreateIndexRequestExt, ListSplitsRequestExt, ListSplitsResponseExt}; use crate::checkpoint::IndexCheckpointDelta; use crate::metastore::instrumented_metastore::InstrumentedMetastore; -use crate::metastore::postgresql_model::{ - DeleteTask as PgDeleteTask, Index as PgIndex, Split as PgSplit, -}; -use crate::metastore::FilterRange; +use crate::metastore::postgresql_model::{PgDeleteTask, PgIndex, PgSplit}; +use crate::metastore::{AddSourceRequestExt, FilterRange, StageSplitsRequestExt}; use crate::{ IndexMetadata, ListSplitsQuery, Metastore, MetastoreError, MetastoreFactory, MetastoreResolverError, MetastoreResult, Split, SplitMaturity, SplitMetadata, SplitState, @@ -228,24 +232,22 @@ fn write_sql_filter( }; } -fn build_query_filter(mut sql: String, query: &ListSplitsQuery) -> String { - sql.push_str(" WHERE index_uid = $1"); +fn build_list_splits_sql_query(query: &ListSplitsQuery) -> String { + let mut sql = "SELECT * FROM splits WHERE index_uid = $1".to_string(); if !query.split_states.is_empty() { let params = query .split_states .iter() - .map(|v| format!("'{}'", v.as_str())) + .map(|split_state| format!("'{}'", split_state.as_str())) .join(", "); let _ = write!(sql, " AND split_state IN ({params})"); } - - if let Some(tags) = query.tags.as_ref() { + if let Some(tags) = &query.tags { sql.push_str(" AND ("); sql.push_str(&tags_filter_expression_helper(tags)); sql.push(')'); } - match query.time_range.start { Bound::Included(v) => { let _ = write!( @@ -274,7 +276,6 @@ fn build_query_filter(mut sql: String, query: &ListSplitsQuery) -> String { } Bound::Unbounded => {} }; - match &query.mature { Bound::Included(evaluation_datetime) => { let _ = write!( @@ -293,7 +294,6 @@ fn build_query_filter(mut sql: String, query: &ListSplitsQuery) -> String { } Bound::Unbounded => {} } - // WARNING: Not SQL injection proof write_sql_filter( &mut sql, @@ -310,15 +310,15 @@ fn build_query_filter(mut sql: String, query: &ListSplitsQuery) -> String { write_sql_filter(&mut sql, "delete_opstamp", &query.delete_opstamp, |val| { val.to_string() }); - - if let Some(limit) = query.limit { - let _ = write!(sql, " LIMIT {limit}"); + if query.sort_by_staleness { + let _ = write!(sql, " ORDER BY delete_opstamp ASC, publish_timestamp ASC"); } - if let Some(offset) = query.offset { let _ = write!(sql, " OFFSET {offset}"); } - + if let Some(limit) = query.limit { + let _ = write!(sql, " LIMIT {limit}"); + } sql } @@ -451,18 +451,29 @@ impl Metastore for PostgresqlMetastore { } #[instrument(skip(self))] - async fn list_indexes_metadatas(&self) -> MetastoreResult> { + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { let pg_indexes = sqlx::query_as::<_, PgIndex>("SELECT * FROM indexes") .fetch_all(&self.connection_pool) .await?; - pg_indexes + let indexes_metadata_json = pg_indexes .into_iter() - .map(|pg_index| pg_index.index_metadata()) - .collect() + .map(|pg_index| pg_index.index_metadata_json) + .collect(); + let response = ListIndexesResponse { + indexes_metadata_json, + }; + Ok(response) } - #[instrument(skip(self), fields(index_id=&index_config.index_id))] - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { + #[instrument(skip(self), fields(index_id=request.index_id))] + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_config = request.deserialize_index_config()?; let index_metadata = IndexMetadata::new(index_config); let index_metadata_json = serde_json::to_string(&index_metadata).map_err(|error| { MetastoreError::JsonSerializeError { @@ -479,38 +490,42 @@ impl Metastore for PostgresqlMetastore { .execute(&self.connection_pool) .await .map_err(|error| convert_sqlx_err(index_metadata.index_id(), error))?; - Ok(index_metadata.index_uid) + + let response = CreateIndexResponse { + index_uid: index_metadata.index_uid.into(), + }; + Ok(response) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { let delete_res = sqlx::query("DELETE FROM indexes WHERE index_uid = $1") - .bind(index_uid.to_string()) + .bind(&request.index_uid) .execute(&self.connection_pool) .await?; if delete_res.rows_affected() == 0 { + let index_uid: IndexUid = request.index_uid.into(); return Err(MetastoreError::IndexDoesNotExist { index_id: index_uid.index_id().to_string(), }); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self, split_metadata_list), fields(split_ids))] - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - let mut split_ids = Vec::with_capacity(split_metadata_list.len()); - let mut time_range_start_list = Vec::with_capacity(split_metadata_list.len()); - let mut time_range_end_list = Vec::with_capacity(split_metadata_list.len()); - let mut tags_list = Vec::with_capacity(split_metadata_list.len()); - let mut split_metadata_json_list = Vec::with_capacity(split_metadata_list.len()); - let mut delete_opstamps = Vec::with_capacity(split_metadata_list.len()); - let mut maturity_timestamps = Vec::with_capacity(split_metadata_list.len()); - - for split_metadata in split_metadata_list { + #[instrument(skip(self, request), fields(split_ids))] + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + let splits_metadata = request.deserialize_splits_metadata()?; + let index_uid: IndexUid = request.index_uid.into(); + + 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 split_metadata_json_list = 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()); + + for split_metadata in splits_metadata { let split_metadata_json = serde_json::to_string(&split_metadata).map_err(|error| { MetastoreError::JsonSerializeError { struct_name: "SplitMetadata".to_string(), @@ -529,8 +544,8 @@ impl Metastore for PostgresqlMetastore { let time_range_end = split_metadata.time_range.map(|range| *range.end()); time_range_end_list.push(time_range_end); - let tags: Vec = split_metadata.tags.into_iter().collect(); - tags_list.push(sqlx::types::Json(tags)); + // let tags: Vec = split_metadata.tags.into_iter().collect(); + tags_list.push(sqlx::types::Json(split_metadata.tags)); split_ids.push(split_metadata.split_id); delete_opstamps.push(split_metadata.delete_opstamp as i64); } @@ -583,27 +598,24 @@ impl Metastore for PostgresqlMetastore { if upserted_split_ids.len() != split_ids.len() { let failed_split_ids = split_ids .into_iter() - .filter(|id| !upserted_split_ids.contains(id)) + .filter(|split_id| !upserted_split_ids.contains(split_id)) .collect(); return Err(MetastoreError::SplitsNotStaged { split_ids: failed_split_ids, }); } - debug!(index_id=%index_uid.index_id(), num_splits=split_ids.len(), "Splits successfully staged."); - - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn publish_splits<'a>( + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn publish_splits( &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { + request: PublishSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + run_with_tx!(self.connection_pool, tx, { let mut index_metadata = index_metadata(tx, index_uid.index_id()).await?; if index_metadata.index_uid != index_uid { @@ -611,7 +623,8 @@ impl Metastore for PostgresqlMetastore { index_id: index_uid.index_id().to_string(), }); } - if let Some(checkpoint_delta) = checkpoint_delta_opt { + if let Some(checkpoint_delta) = request.checkpoint_delta { + let checkpoint_delta = IndexCheckpointDelta::from(checkpoint_delta); index_metadata .checkpoint .try_apply_delta(checkpoint_delta)?; @@ -699,8 +712,8 @@ impl Metastore for PostgresqlMetastore { sqlx::query_as(PUBLISH_SPLITS_QUERY) .bind(index_uid.to_string()) .bind(index_metadata_json) - .bind(staged_split_ids) - .bind(replaced_split_ids) + .bind(request.staged_split_ids) + .bind(request.replaced_split_ids) .fetch_one(tx.as_mut()) .await .map_err(|error| convert_sqlx_err(index_uid.index_id(), error))?; @@ -725,42 +738,45 @@ impl Metastore for PostgresqlMetastore { "Published {} splits and marked {} splits for deletion successfully.", num_published_splits, num_marked_splits ); - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=query.index_uid.index_id()))] - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - let sql_base = "SELECT * FROM splits".to_string(); - let sql = build_query_filter(sql_base, &query); + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { + let list_splits_query = request.deserialize_list_splits_query()?; + let index_uid: IndexUid = request.index_uid.into(); + + let sql = build_list_splits_sql_query(&list_splits_query); let pg_splits = sqlx::query_as::<_, PgSplit>(&sql) - .bind(query.index_uid.to_string()) + .bind(index_uid.to_string()) .fetch_all(&self.connection_pool) .await?; // If no splits were returned, maybe the index does not exist in the first place? if pg_splits.is_empty() - && index_opt_for_uid(&self.connection_pool, query.index_uid.clone()) + && index_opt_for_uid(&self.connection_pool, index_uid.clone()) .await? .is_none() { return Err(MetastoreError::IndexDoesNotExist { - index_id: query.index_uid.index_id().to_string(), + index_id: index_uid.index_id().to_string(), }); } - pg_splits + let splits_metadata: Vec = pg_splits .into_iter() - .map(|pg_split| pg_split.try_into()) - .collect() + .map(|pg_split| pg_split.split_metadata()) + .collect::>()?; + let response = ListSplitsResponse::try_from_splits_metadata(splits_metadata)?; + Ok(response) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn mark_splits_for_deletion<'a>( + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { 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. @@ -796,10 +812,11 @@ impl Metastore for PostgresqlMetastore { COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) FROM input_splits "#; + let index_uid: IndexUid = request.index_uid.into(); 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.to_string()) - .bind(split_ids) + .bind(index_uid.as_str()) + .bind(&request.split_ids) .fetch_one(&self.connection_pool) .await .map_err(|error| convert_sqlx_err(index_uid.index_id(), error))?; @@ -816,7 +833,7 @@ impl Metastore for PostgresqlMetastore { info!( index_id=%index_uid.index_id(), "Marked {} splits for deletion, among which {} were newly marked.", - split_ids.len() - not_found_split_ids.len(), + request.split_ids.len() - not_found_split_ids.len(), num_marked_splits ); if !not_found_split_ids.is_empty() { @@ -827,15 +844,11 @@ impl Metastore for PostgresqlMetastore { not_found_split_ids.len() ); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { 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. @@ -874,14 +887,15 @@ impl Metastore for PostgresqlMetastore { COALESCE(ARRAY_AGG(split_id) FILTER (WHERE split_state IS NULL), ARRAY[]::TEXT[]) FROM input_splits "#; + let index_uid: IndexUid = request.index_uid.into(); 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.to_string()) - .bind(split_ids) + .bind(index_uid.as_str()) + .bind(&request.split_ids) .fetch_one(&self.connection_pool) .await .map_err(|error| convert_sqlx_err(index_uid.index_id(), error))?; @@ -910,74 +924,82 @@ impl Metastore for PostgresqlMetastore { not_found_split_ids.len() ); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_id))] - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - index_opt(&self.connection_pool, index_id) + #[instrument(skip(self), fields(index_id=request.index_id))] + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let pg_index = index_opt(&self.connection_pool, &request.index_id) .await? .ok_or_else(|| MetastoreError::IndexDoesNotExist { - index_id: index_id.to_string(), - })? - .index_metadata() + index_id: request.index_id, + })?; + Ok(IndexMetadataResponse { + index_metadata_json: pg_index.index_metadata_json, + }) } - #[instrument(skip(self, source), fields(index_id=index_uid.index_id(), source_id=source.source_id))] - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + // #[instrument(skip(self, request), fields(index_id=index_uid.index_id(), + // source_id=source.source_id))] + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid = request.index_uid.into(); + run_with_tx!(self.connection_pool, tx, { mutate_index_metadata::( tx, index_uid, |index_metadata: &mut IndexMetadata| { - index_metadata.add_source(source)?; + index_metadata.add_source(source_config)?; Ok(true) }, ) .await?; - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + #[instrument(skip(self), fields(index_id=request.index_uid, source_id=request.source_id))] // FIXME + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - index_metadata.toggle_source(source_id, enable) + index_metadata.toggle_source(&request.source_id, request.enable) }) .await?; - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + #[instrument(skip(self), fields(index_id=request.index_uid, source_id=request.source_id))] // FIXME + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - index_metadata.delete_source(source_id) + index_metadata.delete_source(&request.source_id) }) .await?; - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] + #[instrument(skip(self), fields(index_id=request.index_uid, source_id=request.source_id))] // FIXME async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - Ok::<_, MetastoreError>(index_metadata.checkpoint.reset_source(source_id)) + Ok::<_, MetastoreError>(index_metadata.checkpoint.reset_source(&request.source_id)) }) .await?; - Ok(()) + Ok(EmptyResponse {}) }) } @@ -1040,16 +1062,15 @@ impl Metastore for PostgresqlMetastore { } /// Update splits delete opstamps. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn update_splits_delete_opstamp<'a>( + #[instrument(skip(self), fields(index_id=request.index_uid))] + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - if split_ids.is_empty() { - return Ok(()); + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + if request.split_ids.is_empty() { + return Ok(EmptyResponse {}); } + let index_uid: IndexUid = request.index_uid.clone().into(); let update_res = sqlx::query( r#" UPDATE splits @@ -1065,9 +1086,9 @@ impl Metastore for PostgresqlMetastore { AND split_id = ANY($3) "#, ) - .bind(delete_opstamp as i64) - .bind(index_uid.to_string()) - .bind(split_ids) + .bind(request.delete_opstamp as i64) + .bind(request.index_uid) + .bind(request.split_ids) .execute(&self.connection_pool) .await?; @@ -1081,16 +1102,15 @@ impl Metastore for PostgresqlMetastore { index_id: index_uid.index_id().to_string(), }); } - Ok(()) + Ok(EmptyResponse {}) } /// Lists the delete tasks with opstamp > `opstamp_start`. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] + #[instrument(skip(self), fields(index_id=request.index_uid))] async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + request: ListDeleteTasksRequest, + ) -> MetastoreResult { let pg_delete_tasks: Vec = sqlx::query_as::<_, PgDeleteTask>( r#" SELECT * FROM delete_tasks @@ -1099,60 +1119,16 @@ impl Metastore for PostgresqlMetastore { AND opstamp > $2 "#, ) - .bind(index_uid.to_string()) - .bind(opstamp_start as i64) + .bind(request.index_uid) + .bind(request.opstamp_start as i64) .fetch_all(&self.connection_pool) .await?; - pg_delete_tasks + let delete_tasks: Vec = pg_delete_tasks .into_iter() .map(|pg_delete_task| pg_delete_task.try_into()) - .collect() - } - - /// 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), fields(index_id=index_uid.index_id()))] - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - let pg_stale_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.to_string()) - .bind(delete_opstamp as i64) - .bind(SplitState::Published.as_str()) - .bind(num_splits as i64) - .fetch_all(&self.connection_pool) - .await?; - - // If no splits were returned, maybe the index does not exist in the first place? - if pg_stale_splits.is_empty() - && index_opt_for_uid(&self.connection_pool, index_uid.clone()) - .await? - .is_none() - { - return Err(MetastoreError::IndexDoesNotExist { - index_id: index_uid.index_id().to_string(), - }); - } - pg_stale_splits - .into_iter() - .map(|pg_split| pg_split.try_into()) - .collect() + .collect::>()?; + let response = ListDeleteTasksResponse { delete_tasks }; + Ok(response) } } @@ -1162,11 +1138,12 @@ impl Metastore for PostgresqlMetastore { // we need to generate a string that does not appear in // the literal we want to dollar quote. fn generate_dollar_guard(s: &str) -> String { + let mut dollar_guard = String::new(); + if !s.contains('$') { // That's our happy path here. - return String::new(); + return dollar_guard; } - let mut dollar_guard = String::new(); loop { dollar_guard.push_str("Quickwit!"); // This terminates because `dollar_guard` @@ -1313,7 +1290,7 @@ mod tests { use quickwit_proto::IndexUid; use time::OffsetDateTime; - use super::{build_query_filter, tags_filter_expression_helper}; + use super::{build_list_splits_sql_query, tags_filter_expression_helper}; use crate::{ListSplitsQuery, SplitState}; fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: &str) { @@ -1378,82 +1355,78 @@ mod tests { #[test] fn test_single_sql_query_builder() { let index_uid = IndexUid::new("test-index"); - let query = - ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_split_state(SplitState::Staged); + let sql = build_list_splits_sql_query(&query); assert_eq!(sql, " WHERE index_uid = $1 AND split_state IN ('Staged')"); - let query = - ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_split_state(SplitState::Published); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND split_state IN ('Published')" ); - let query = ListSplitsQuery::for_index(index_uid.clone()) + let query = ListSplitsQuery::default() .with_split_states([SplitState::Published, SplitState::MarkedForDeletion]); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND split_state IN ('Published', 'MarkedForDeletion')" ); - let query = ListSplitsQuery::for_index(index_uid.clone()).with_update_timestamp_lt(51); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_update_timestamp_lt(51); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND update_timestamp < to_timestamp(51)" ); - let query = ListSplitsQuery::for_index(index_uid.clone()).with_create_timestamp_lte(55); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_create_timestamp_lte(55); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND create_timestamp <= to_timestamp(55)" ); let maturity_evaluation_datetime = OffsetDateTime::from_unix_timestamp(55).unwrap(); - let query = ListSplitsQuery::for_index(index_uid.clone()) - .retain_mature(maturity_evaluation_datetime); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().retain_mature(maturity_evaluation_datetime); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (maturity_timestamp = to_timestamp(0) OR to_timestamp(55) \ >= maturity_timestamp)" ); - let query = ListSplitsQuery::for_index(index_uid.clone()) - .retain_immature(maturity_evaluation_datetime); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().retain_immature(maturity_evaluation_datetime); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND to_timestamp(55) < maturity_timestamp" ); - let query = ListSplitsQuery::for_index(index_uid.clone()).with_delete_opstamp_gte(4); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_delete_opstamp_gte(4); + let sql = build_list_splits_sql_query(&query); assert_eq!(sql, " WHERE index_uid = $1 AND delete_opstamp >= 4"); - let query = ListSplitsQuery::for_index(index_uid.clone()).with_time_range_start_gt(45); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_time_range_start_gt(45); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (time_range_end > 45 OR time_range_end IS NULL)" ); - let query = ListSplitsQuery::for_index(index_uid.clone()).with_time_range_end_lt(45); - let sql = build_query_filter(String::new(), &query); + let query = ListSplitsQuery::default().with_time_range_end_lt(45); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (time_range_start < 45 OR time_range_start IS NULL)" ); - let query = ListSplitsQuery::for_index(index_uid).with_tags_filter(TagFilterAst::Tag { + let query = ListSplitsQuery::default().with_tags_filter(TagFilterAst::Tag { is_present: false, tag: "tag-2".to_string(), }); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (NOT ($$tag-2$$ = ANY(tags)))" @@ -1463,43 +1436,43 @@ mod tests { #[test] fn test_combination_sql_query_builder() { let index_uid = IndexUid::new("test-index"); - let query = ListSplitsQuery::for_index(index_uid.clone()) + let query = ListSplitsQuery::default() .with_time_range_start_gt(0) .with_time_range_end_lt(40); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (time_range_end > 0 OR time_range_end IS NULL) AND \ (time_range_start < 40 OR time_range_start IS NULL)" ); - let query = ListSplitsQuery::for_index(index_uid.clone()) + let query = ListSplitsQuery::default() .with_time_range_start_gt(45) .with_delete_opstamp_gt(0); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND (time_range_end > 45 OR time_range_end IS NULL) AND \ delete_opstamp > 0" ); - let query = ListSplitsQuery::for_index(index_uid.clone()) + let query = ListSplitsQuery::default() .with_update_timestamp_lt(51) .with_create_timestamp_lte(63); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND update_timestamp < to_timestamp(51) AND create_timestamp \ <= to_timestamp(63)" ); - let query = ListSplitsQuery::for_index(index_uid) + let query = ListSplitsQuery::default() .with_time_range_start_gt(90) .with_tags_filter(TagFilterAst::Tag { is_present: true, tag: "tag-1".to_string(), }); - let sql = build_query_filter(String::new(), &query); + let sql = build_list_splits_sql_query(&query); assert_eq!( sql, " WHERE index_uid = $1 AND ($$tag-1$$ = ANY(tags)) AND (time_range_end > 90 OR \ diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs index ae5aca36c13..39e7b5d9f85 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs @@ -20,18 +20,15 @@ use std::convert::TryInto; use std::str::FromStr; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask as QuickwitDeleteTask}; +use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; use quickwit_proto::IndexUid; use tracing::error; -use crate::{ - IndexMetadata, MetastoreError, MetastoreResult, Split as QuickwitSplit, SplitMetadata, - SplitState, -}; +use crate::{IndexMetadata, MetastoreError, MetastoreResult, Split, SplitMetadata, SplitState}; /// A model structure for handling index metadata in a database. #[derive(sqlx::FromRow)] -pub struct Index { +pub(super) struct PgIndex { /// Index UID. The index UID identifies the index when querying the metastore from the /// application. #[sqlx(try_from = "String")] @@ -44,9 +41,7 @@ pub struct Index { pub create_timestamp: sqlx::types::time::PrimitiveDateTime, } -impl Index { - /// Deserializes index metadata from JSON string stored in column and sets appropriate - /// timestamps. +impl PgIndex { pub fn index_metadata(&self) -> MetastoreResult { let mut index_metadata = serde_json::from_str::(&self.index_metadata_json) .map_err(|error| { @@ -57,9 +52,6 @@ impl Index { message: error.to_string(), } })?; - // `create_timestamp` and `update_timestamp` are stored in dedicated columns but are also - // duplicated in [`IndexMetadata`]. We must override the duplicates with the authentic - // values upon deserialization. index_metadata.create_timestamp = self.create_timestamp.assume_utc().unix_timestamp(); Ok(index_metadata) } @@ -67,7 +59,7 @@ impl Index { /// A model structure for handling split metadata in a database. #[derive(sqlx::FromRow)] -pub struct Split { +pub(super) struct PgSplit { /// Split ID. pub split_id: String, /// The state of the split. With `update_timestamp`, this is the only mutable attribute of the @@ -97,17 +89,23 @@ pub struct Split { pub delete_opstamp: i64, } -impl Split { +impl PgSplit { /// Deserializes and returns the split's metadata. - fn split_metadata(&self) -> MetastoreResult { - serde_json::from_str::(&self.split_metadata_json).map_err(|error| { + pub fn split_metadata(&self) -> MetastoreResult { + let mut split_metadata = serde_json::from_str::(&self.split_metadata_json).map_err(|error| { error!(index_id=%self.index_uid.index_id(), split_id=%self.split_id, error=?error, "Failed to deserialize split metadata."); MetastoreError::JsonDeserializeError { struct_name: "SplitMetadata".to_string(), message: error.to_string(), } - }) + })?; + // `create_timestamp` and `delete_opstamp` are duplicated in `SplitMetadata` and needs to be + // overridden with the "true" value stored in a column. + split_metadata.create_timestamp = self.create_timestamp.assume_utc().unix_timestamp(); + split_metadata.delete_opstamp = self.delete_opstamp as u64; + + Ok(split_metadata) } /// Deserializes and returns the split's state. @@ -123,33 +121,29 @@ impl Split { } } -impl TryInto for Split { +impl TryInto for PgSplit { type Error = MetastoreError; - fn try_into(self) -> Result { + fn try_into(self) -> Result { let mut split_metadata = self.split_metadata()?; - // `create_timestamp` and `delete_opstamp` are duplicated in `SplitMetadata` and needs to be - // overridden with the "true" value stored in a column. - split_metadata.create_timestamp = self.create_timestamp.assume_utc().unix_timestamp(); let split_state = self.split_state()?; let update_timestamp = self.update_timestamp.assume_utc().unix_timestamp(); let publish_timestamp = self .publish_timestamp .map(|publish_timestamp| publish_timestamp.assume_utc().unix_timestamp()); - split_metadata.index_uid = self.index_uid; - split_metadata.delete_opstamp = self.delete_opstamp as u64; - Ok(QuickwitSplit { + let split = Split { split_metadata, split_state, update_timestamp, publish_timestamp, - }) + }; + Ok(split) } } /// A model structure for handling split metadata in a database. #[derive(sqlx::FromRow)] -pub struct DeleteTask { +pub(super) struct PgDeleteTask { /// Create timestamp. pub create_timestamp: sqlx::types::time::PrimitiveDateTime, /// Monotonic increasing unique opstamp. @@ -161,7 +155,7 @@ pub struct DeleteTask { pub delete_query_json: String, } -impl DeleteTask { +impl PgDeleteTask { /// Deserializes and returns the split's metadata. fn delete_query(&self) -> MetastoreResult { serde_json::from_str::(&self.delete_query_json).map_err(|error| { @@ -175,15 +169,16 @@ impl DeleteTask { } } -impl TryInto for DeleteTask { +impl TryInto for PgDeleteTask { type Error = MetastoreError; - fn try_into(self) -> Result { + fn try_into(self) -> Result { let delete_query = self.delete_query()?; - Ok(QuickwitDeleteTask { + let delete_task = DeleteTask { create_timestamp: self.create_timestamp.assume_utc().unix_timestamp(), opstamp: self.opstamp as u64, delete_query: Some(delete_query), - }) + }; + Ok(delete_task) } } diff --git a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs index 7c99545ec25..c705d206d7c 100644 --- a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs @@ -24,13 +24,18 @@ mod test; use async_trait::async_trait; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use self::retry::{retry, RetryParams}; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListSplitsQuery, Metastore, MetastoreResult, Split, SplitMetadata}; +use crate::{Metastore, MetastoreResult}; /// Retry layer for a [`Metastore`]. /// This is a band-aid solution for now. This will be removed after retry can be usable on @@ -64,165 +69,120 @@ impl Metastore for RetryingMetastore { self.inner.check_connectivity().await } - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { + async fn create_index( + &self, + request: CreateIndexRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.create_index(index_config.clone()).await + self.inner.create_index(request.clone()).await }) .await } - async fn index_exists(&self, index_id: &str) -> MetastoreResult { + async fn index_metadata( + &self, + request: IndexMetadataRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.index_exists(index_id).await + self.inner.index_metadata(request.clone()).await }) .await } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { + async fn list_indexes( + &self, + request: ListIndexesRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.index_metadata(index_id).await + self.inner.list_indexes(request.clone()).await }) .await } - async fn list_indexes_metadatas(&self) -> MetastoreResult> { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.list_indexes_metadatas().await + self.inner.delete_index(request.clone()).await }) .await } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.delete_index(index_uid.clone()).await + self.inner.stage_splits(request.clone()).await }) .await } - async fn stage_splits( + async fn publish_splits( &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { + request: PublishSplitsRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .stage_splits(index_uid.clone(), split_metadata_list.clone()) - .await + self.inner.publish_splits(request.clone()).await }) .await } - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { + async fn list_splits(&self, request: ListSplitsRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .publish_splits( - index_uid.clone(), - split_ids, - replaced_split_ids, - checkpoint_delta_opt.clone(), - ) - .await + self.inner.list_splits(request.clone()).await }) .await } - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner.list_splits(query.clone()).await - }) - .await - } + // async fn list_stale_splits( + // &self, + // index_uid: IndexUid, + // delete_opstamp: u64, + // num_splits: usize, + // ) -> MetastoreResult> { retry(&self.retry_params, || async { self.inner + // .list_stale_splits(index_uid.clone(), delete_opstamp, num_splits) .await }) .await + // } - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner.list_all_splits(index_uid.clone()).await - }) - .await - } - - async fn list_stale_splits( + async fn mark_splits_for_deletion( &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .list_stale_splits(index_uid.clone(), delete_opstamp, num_splits) - .await + self.inner.mark_splits_for_deletion(request.clone()).await }) .await } - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .mark_splits_for_deletion(index_uid.clone(), split_ids) - .await - }) - .await - } - - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.delete_splits(index_uid.clone(), split_ids).await + self.inner.delete_splits(request.clone()).await }) .await } - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .add_source(index_uid.clone(), source.clone()) - .await + self.inner.add_source(request.clone()).await }) .await } - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .toggle_source(index_uid.clone(), source_id, enable) - .await + self.inner.toggle_source(request.clone()).await }) .await } async fn reset_source_checkpoint( &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .reset_source_checkpoint(index_uid.clone(), source_id) - .await + self.inner.reset_source_checkpoint(request.clone()).await }) .await } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner.delete_source(index_uid.clone(), source_id).await + self.inner.delete_source(request.clone()).await }) .await } @@ -241,15 +201,13 @@ impl Metastore for RetryingMetastore { .await } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { self.inner - .update_splits_delete_opstamp(index_uid.clone(), split_ids, delete_opstamp) + .update_splits_delete_opstamp(request.clone()) .await }) .await @@ -257,13 +215,10 @@ impl Metastore for RetryingMetastore { async fn list_delete_tasks( &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + request: ListDeleteTasksRequest, + ) -> MetastoreResult { retry(&self.retry_params, || async { - self.inner - .list_delete_tasks(index_uid.clone(), opstamp_start) - .await + self.inner.list_delete_tasks(request.clone()).await }) .await } diff --git a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs index 857dfb5c553..aa26dd28046 100644 --- a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs +++ b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs @@ -21,16 +21,18 @@ use std::sync::atomic::{AtomicUsize, Ordering}; use async_trait::async_trait; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, IndexMetadataRequest, + IndexMetadataResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesRequest, + ListIndexesResponse, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::IndexUid; use super::retry::RetryParams; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{ - IndexMetadata, ListSplitsQuery, Metastore, MetastoreError, MetastoreResult, RetryingMetastore, - Split, SplitMetadata, -}; +use crate::{Metastore, MetastoreError, MetastoreResult, RetryingMetastore}; struct RetryTestMetastore { retry_count: AtomicUsize, @@ -81,99 +83,90 @@ impl Metastore for RetryTestMetastore { self.try_success().map_err(anyhow::Error::from) } - async fn create_index(&self, _index_config: IndexConfig) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(IndexUid::new("")), - Err(err) => Err(err), - } + async fn create_index( + &self, + _request: CreateIndexRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(CreateIndexResponse::default()) } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(IndexMetadata::for_test(index_id, "")), - Err(err) => Err(err), - } + async fn index_metadata( + &self, + _request: IndexMetadataRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(IndexMetadataResponse::default()) } - async fn list_indexes_metadatas(&self) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } + async fn list_indexes( + &self, + _request: ListIndexesRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(ListIndexesResponse::default()) } - async fn delete_index(&self, _index_uid: IndexUid) -> MetastoreResult<()> { - self.try_success() + async fn delete_index(&self, _request: DeleteIndexRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn stage_splits( - &self, - _index_uid: IndexUid, - _split_metadata_list: Vec, - ) -> MetastoreResult<()> { - self.try_success() + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn publish_splits<'a>( + async fn publish_splits( &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - _replaced_split_ids: &[&'a str], - _checkpoint_delta_opt: Option, - ) -> MetastoreResult<()> { - self.try_success() + _request: PublishSplitsRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn list_splits(&self, _query: ListSplitsQuery) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } + async fn list_splits( + &self, + _request: ListSplitsRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(ListSplitsResponse::default()) } - async fn mark_splits_for_deletion<'a>( + async fn mark_splits_for_deletion( &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.try_success() + _request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn delete_splits<'a>( - &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.try_success() + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn add_source(&self, _index_uid: IndexUid, _source: SourceConfig) -> MetastoreResult<()> { - self.try_success() + async fn add_source(&self, _request: AddSourceRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn toggle_source( - &self, - _index_uid: IndexUid, - _source_id: &str, - _enable: bool, - ) -> MetastoreResult<()> { - self.try_success() + async fn toggle_source(&self, _request: ToggleSourceRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } async fn reset_source_checkpoint( &self, - _index_uid: IndexUid, - _source_id: &str, - ) -> MetastoreResult<()> { - self.try_success() + _request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } - async fn delete_source(&self, _index_uid: IndexUid, _source_id: &str) -> MetastoreResult<()> { - self.try_success() + async fn delete_source(&self, _request: DeleteSourceRequest) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } async fn create_delete_task(&self, _delete_query: DeleteQuery) -> MetastoreResult { @@ -196,25 +189,20 @@ impl Metastore for RetryTestMetastore { } } - async fn update_splits_delete_opstamp<'a>( + async fn update_splits_delete_opstamp( &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - _delete_opstamp: u64, - ) -> MetastoreResult<()> { - self.try_success() + _request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(EmptyResponse {}) } async fn list_delete_tasks( &self, - _index_uid: IndexUid, - _opstamp_start: u64, - ) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } + _request: ListDeleteTasksRequest, + ) -> MetastoreResult { + self.try_success()?; + Ok(ListDeleteTasksResponse::default()) } } @@ -241,7 +229,7 @@ async fn test_retryable_metastore_errors() { // On retryable errors, if max retry count is not achieved, RetryingMetastore should retry until // success - assert!(metastore.list_indexes_metadatas().await.is_ok()); + assert!(metastore.list_indexes(ListIndexesRequest {}).await.is_ok()); let metastore: RetryingMetastore = RetryTestMetastore::new_retrying_with_errors( 5, @@ -251,7 +239,7 @@ async fn test_retryable_metastore_errors() { ); // On non-retryable errors, RetryingMetastore should exit with an error. - assert!(metastore.list_indexes_metadatas().await.is_err()); + assert!(metastore.list_indexes(ListIndexesRequest {}).await.is_err()); } #[tokio::test] @@ -267,7 +255,10 @@ async fn test_retryable_more_than_max_retry() { .collect::>(), ); - let error = metastore.list_indexes_metadatas().await.unwrap_err(); + let error = metastore + .list_indexes(ListIndexesRequest {}) + .await + .unwrap_err(); assert_eq!( error, MetastoreError::ConnectionError { @@ -299,7 +290,10 @@ async fn test_mixed_retryable_metastore_errors() { ], ); - let error = metastore.list_indexes_metadatas().await.unwrap_err(); + let error = metastore + .list_indexes(ListIndexesRequest {}) + .await + .unwrap_err(); assert_eq!( error, diff --git a/quickwit/quickwit-metastore/src/tests.rs b/quickwit/quickwit-metastore/src/tests.rs index 4a37343d06f..f70eed96446 100644 --- a/quickwit/quickwit-metastore/src/tests.rs +++ b/quickwit/quickwit-metastore/src/tests.rs @@ -31,7 +31,11 @@ pub mod test_suite { use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; use quickwit_proto::metastore::DeleteQuery; - use quickwit_proto::IndexUid; + use quickwit_proto::{ + DeleteIndexRequest, DeleteSourceRequest, DeleteSplitsRequest, IndexMetadataRequest, + IndexUid, ListSplitsRequest, MarkSplitsForDeletionRequest, ResetSourceCheckpointRequest, + ToggleSourceRequest, + }; use quickwit_query::query_ast::qast_helper; use time::OffsetDateTime; use tokio::time::sleep; @@ -40,8 +44,10 @@ pub mod test_suite { use crate::checkpoint::{ IndexCheckpointDelta, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, }; + use crate::metastore::ListSplitsResponseExt; use crate::{ - ListSplitsQuery, Metastore, MetastoreError, Split, SplitMaturity, SplitMetadata, SplitState, + IndexMetadata, ListSplitsQuery, Metastore, MetastoreError, Split, SplitMaturity, + SplitMetadata, SplitState, }; #[async_trait] @@ -63,26 +69,34 @@ pub mod test_suite { async fn cleanup_index(metastore: &dyn Metastore, index_uid: IndexUid) { // List all splits. - let all_splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let response = metastore + .list_splits(ListSplitsRequest::all(index_uid.clone())) + .await + .unwrap(); + let all_splits = response.deserialize_splits_metadata().unwrap(); if !all_splits.is_empty() { let all_split_ids: Vec<&str> = all_splits.iter().map(|split| split.split_id()).collect(); // Mark splits for deletion. + let mark_splits_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), all_split_ids); metastore - .mark_splits_for_deletion(index_uid.clone(), &all_split_ids) + .mark_splits_for_deletion(mark_splits_request) .await .unwrap(); // Delete splits. + let delete_splits_request = DeleteSplitsRequest::new(index_uid.clone(), all_split_ids); metastore - .delete_splits(index_uid.clone(), &all_split_ids) + .delete_splits(delete_splits_request) .await .unwrap(); } // Delete index. - metastore.delete_index(index_uid).await.unwrap(); + let delete_index_request = DeleteIndexRequest::new(index_uid); + metastore.delete_index(delete_index_request).await.unwrap(); } // Index API tests @@ -104,7 +118,13 @@ pub mod test_suite { assert!(metastore.index_exists(&index_id).await.unwrap()); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); assert_eq!(index_metadata.index_id(), index_id); assert_eq!(index_metadata.index_uri(), &index_uri); @@ -165,7 +185,13 @@ pub mod test_suite { let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); assert_eq!(index_metadata.index_id(), index_id); assert_eq!(index_metadata.index_uri(), &index_uri); @@ -217,21 +243,24 @@ pub mod test_suite { let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - let error = metastore - .delete_index(IndexUid::new("index-not-found")) - .await - .unwrap_err(); + let request = DeleteIndexRequest { + index_uid: "index-not-found".to_string(), + }; + let error = metastore.delete_index(request).await.unwrap_err(); assert!(matches!(error, MetastoreError::IndexDoesNotExist { .. })); - let error = metastore - .delete_index(IndexUid::new("test-delete-index")) - .await - .unwrap_err(); + let request = DeleteIndexRequest { + index_uid: "index-not-found".to_string(), + }; + let error = metastore.delete_index(request).await.unwrap_err(); assert!(matches!(error, MetastoreError::IndexDoesNotExist { .. })); let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - metastore.delete_index(index_uid.clone()).await.unwrap(); + let request = DeleteIndexRequest { + index_uid: index_uid.clone().into(), + }; + metastore.delete_index(request).await.unwrap(); assert!(!metastore.index_exists(&index_id).await.unwrap()); @@ -253,7 +282,8 @@ pub mod test_suite { // a default behavior. Let's implement the logic that allows this test to pass. // let error = metastore.delete_index(index_uid).await.unwrap_err(); // assert!(matches!(error, MetastoreError::IndexNotEmpty { .. })); - // let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + // let splits = + // metastore.list_splits(ListSplitsRequest::all(index_uid.clone())).await.unwrap(); // assert_eq!(splits.len(), 1) cleanup_index(&metastore, index_uid).await; @@ -295,7 +325,13 @@ pub mod test_suite { .await .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); let sources = &index_metadata.sources; assert_eq!(sources.len(), 1); @@ -354,25 +390,47 @@ pub mod test_suite { .add_source(index_uid.clone(), source.clone()) .await .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, true); // Disable source. - metastore - .toggle_source(index_uid.clone(), &source.source_id, false) + let request = ToggleSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source.source_id.clone(), + enable: false, + }; + metastore.toggle_source(request).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) .await + .unwrap() + .try_into() .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, false); // Enable source. - metastore - .toggle_source(index_uid.clone(), &source.source_id, true) + let request = ToggleSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source.source_id.clone(), + enable: true, + }; + metastore.toggle_source(request).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) .await + .unwrap() + .try_into() .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, true); @@ -418,33 +476,39 @@ pub mod test_suite { .add_source(index_uid.clone(), source) .await .unwrap(); - metastore - .delete_source(index_uid.clone(), &source_id) - .await - .unwrap(); + let request = DeleteSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + }; + metastore.delete_source(request).await.unwrap(); let sources = metastore.index_metadata(&index_id).await.unwrap().sources; assert!(sources.is_empty()); + let request = DeleteSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + }; assert!(matches!( - metastore - .delete_source(index_uid.clone(), &source_id) - .await - .unwrap_err(), + metastore.delete_source(request).await.unwrap_err(), MetastoreError::SourceDoesNotExist { .. } )); + + let request = DeleteSourceRequest { + index_uid: "index-not-found".to_string(), + source_id: source_id.clone(), + }; assert!(matches!( - metastore - .delete_source(IndexUid::new("index-not-found"), &source_id) - .await - .unwrap_err(), + metastore.delete_source(request).await.unwrap_err(), MetastoreError::IndexDoesNotExist { .. } )); + + let request = DeleteSourceRequest { + index_uid: index_id.clone().into(), + source_id: source_id.clone(), + }; assert!(matches!( - metastore - .delete_source(IndexUid::new(index_id), &source_id) - .await - .unwrap_err(), + metastore.delete_source(request).await.unwrap_err(), MetastoreError::IndexDoesNotExist { .. } )); @@ -499,12 +563,19 @@ pub mod test_suite { .checkpoint .is_empty()); - metastore - .reset_source_checkpoint(index_uid.clone(), &source_ids[0]) + let request = ResetSourceCheckpointRequest { + index_uid: index_uid.clone().into(), + source_id: source_ids[0].clone(), + }; + metastore.reset_source_checkpoint(request).await.unwrap(); + + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) .await + .unwrap() + .try_into() .unwrap(); - - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); assert!(index_metadata .checkpoint .source_checkpoint(&source_ids[0]) @@ -515,26 +586,35 @@ pub mod test_suite { .source_checkpoint(&source_ids[1]) .is_some()); + let request = ResetSourceCheckpointRequest { + index_uid: "index-not-found".to_string(), + source_id: source_ids[1].clone(), + }; assert!(matches!( metastore - .reset_source_checkpoint(IndexUid::new("index-not-found"), &source_ids[1]) + .reset_source_checkpoint(request) .await .unwrap_err(), MetastoreError::IndexDoesNotExist { .. } )); + let request = ResetSourceCheckpointRequest { + index_uid: index_id.clone().into(), + source_id: source_ids[1].clone(), + }; assert!(matches!( metastore - .reset_source_checkpoint(IndexUid::new(&index_id), &source_ids[1]) + .reset_source_checkpoint(request) .await .unwrap_err(), MetastoreError::IndexDoesNotExist { .. } )); - metastore - .reset_source_checkpoint(index_uid.clone(), &source_ids[1]) - .await - .unwrap(); + let request = ResetSourceCheckpointRequest { + index_uid: index_uid.clone().into(), + source_id: source_ids[1].clone(), + }; + metastore.reset_source_checkpoint(request).await.unwrap(); assert!(metastore .index_metadata(&index_id) @@ -594,7 +674,13 @@ pub mod test_suite { .await .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); let source_checkpoint = index_metadata .checkpoint .source_checkpoint(&source_id) @@ -1091,7 +1177,13 @@ pub mod test_suite { } try_join_all(join_handles).await.unwrap(); - let index_metadata = metastore.index_metadata(&index_id).await.unwrap(); + let request = IndexMetadataRequest::new(&index_id); + let index_metadata: IndexMetadata = metastore + .index_metadata(request) + .await + .unwrap() + .try_into() + .unwrap(); let source_checkpoint = index_metadata .checkpoint .source_checkpoint(&source_id) @@ -1521,7 +1613,7 @@ pub mod test_suite { assert_eq!( metastore - .list_all_splits(index_uid.clone()) + .list_splits(ListSplitsRequest::all(index_uid.clone())) .await .unwrap() .len(), @@ -1543,7 +1635,7 @@ pub mod test_suite { assert_eq!( metastore - .list_all_splits(index_uid.clone()) + .list_splits(ListSplitsRequest::all(index_uid.clone())) .await .unwrap() .len(), @@ -1631,7 +1723,10 @@ pub mod test_suite { .await .unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::all(index_uid.clone())) + .await + .unwrap(); let split_ids = collect_split_ids(&splits); assert_eq!( split_ids, @@ -2136,7 +2231,11 @@ pub mod test_suite { .unwrap(); sleep(Duration::from_secs(1)).await; - let split_meta = metastore.list_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + let split_meta = metastore + .list_splits(ListSplitsRequest::all(index_uid.clone())) + .await + .unwrap()[0] + .clone(); assert!(split_meta.update_timestamp > current_timestamp); assert!(split_meta.publish_timestamp.is_none()); @@ -2157,7 +2256,11 @@ pub mod test_suite { ) .await .unwrap(); - let split_meta = metastore.list_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + let split_meta = metastore + .list_splits(ListSplitsRequest::all(index_uid.clone())) + .await + .unwrap()[0] + .clone(); assert!(split_meta.update_timestamp > current_timestamp); assert_eq!( split_meta.publish_timestamp, @@ -2171,7 +2274,11 @@ pub mod test_suite { .mark_splits_for_deletion(index_uid.clone(), &[&split_id]) .await .unwrap(); - let split_meta = metastore.list_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + let split_meta = metastore + .list_splits(ListSplitsRequest::all(index_uid.clone())) + .await + .unwrap()[0] + .clone(); assert!(split_meta.update_timestamp > current_timestamp); assert!(split_meta.publish_timestamp.is_some()); @@ -2317,7 +2424,10 @@ pub mod test_suite { .await .unwrap(); - metastore.delete_index(index_uid).await.unwrap(); + let request = DeleteIndexRequest { + index_uid: index_uid.into(), + }; + metastore.delete_index(request).await.unwrap(); } pub async fn test_metastore_list_delete_tasks() { diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index c6f3caf525b..0a619d3a91a 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -30,7 +30,13 @@ fn main() -> Result<(), Box> { .collect(); let mut prost_config = prost_build::Config::default(); - prost_config.protoc_arg("--experimental_allow_proto3_optional"); + prost_config + .bytes(&[ + "IndexMetadataResponse.index_metadata_json", + "ListIndexesResponse.indexes_metadata_json", + "StageSplitsRequest.splits_metadata_json", + ]) + .protoc_arg("--experimental_allow_proto3_optional"); tonic_build::configure() .type_attribute(".", "#[derive(Serialize, Deserialize, utoipa::ToSchema)]") diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 03c263b6571..165db66388f 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -30,40 +30,37 @@ service MetastoreService { rpc index_metadata(IndexMetadataRequest) returns (IndexMetadataResponse); // Gets an indexes metadatas. - rpc list_indexes_metadatas(ListIndexesMetadatasRequest) returns (ListIndexesMetadatasResponse); + rpc list_indexes(ListIndexesRequest) returns (ListIndexesResponse); // Deletes an index - rpc delete_index(DeleteIndexRequest) returns (DeleteIndexResponse); - - // Gets all splits from index. - rpc list_all_splits(ListAllSplitsRequest) returns (ListSplitsResponse); + rpc delete_index(DeleteIndexRequest) returns (EmptyResponse); // Gets splits from index. rpc list_splits(ListSplitsRequest) returns (ListSplitsResponse); // Stages several splits. - rpc stage_splits(StageSplitsRequest) returns (SplitResponse); + rpc stage_splits(StageSplitsRequest) returns (EmptyResponse); // Publishes split. - rpc publish_splits(PublishSplitsRequest) returns (SplitResponse); + rpc publish_splits(PublishSplitsRequest) returns (EmptyResponse); // Marks splits for deletion. - rpc mark_splits_for_deletion(MarkSplitsForDeletionRequest) returns (SplitResponse); + rpc mark_splits_for_deletion(MarkSplitsForDeletionRequest) returns (EmptyResponse); // Deletes splits. - rpc delete_splits(DeleteSplitsRequest) returns (SplitResponse); + rpc delete_splits(DeleteSplitsRequest) returns (EmptyResponse); // Adds source. - rpc add_source(AddSourceRequest) returns (SourceResponse); + rpc add_source(AddSourceRequest) returns (EmptyResponse); // Toggles source. - rpc toggle_source(ToggleSourceRequest) returns (SourceResponse); + rpc toggle_source(ToggleSourceRequest) returns (EmptyResponse); // Removes source. - rpc delete_source(DeleteSourceRequest) returns (SourceResponse); + rpc delete_source(DeleteSourceRequest) returns (EmptyResponse); // Resets source checkpoint. - rpc reset_source_checkpoint(ResetSourceCheckpointRequest) returns (SourceResponse); + rpc reset_source_checkpoint(ResetSourceCheckpointRequest) returns (EmptyResponse); // Gets last opstamp for a given `index_id`. rpc last_delete_opstamp(LastDeleteOpstampRequest) returns (LastDeleteOpstampResponse); @@ -72,67 +69,77 @@ service MetastoreService { rpc create_delete_task(DeleteQuery) returns (DeleteTask); // Updates splits `delete_opstamp`. - rpc update_splits_delete_opstamp(UpdateSplitsDeleteOpstampRequest) returns (UpdateSplitsDeleteOpstampResponse); + rpc update_splits_delete_opstamp(UpdateSplitsDeleteOpstampRequest) returns (EmptyResponse); // Lists delete tasks with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. rpc list_delete_tasks(ListDeleteTasksRequest) returns (ListDeleteTasksResponse); - - /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. - rpc list_stale_splits(ListStaleSplitsRequest) returns (ListSplitsResponse); } +message EmptyResponse {} + message CreateIndexRequest { - string index_config_serialized_json = 2; + string index_id = 1; + string index_config_json = 2; } message CreateIndexResponse { string index_uid = 1; } -message ListIndexesMetadatasRequest {} - -message ListIndexesMetadatasResponse { - string indexes_metadatas_serialized_json = 1; -} - -message DeleteIndexRequest { - string index_uid = 1; -} - -message DeleteTasksResponse {} - -message DeleteIndexResponse {} - message IndexMetadataRequest { string index_id = 1; + optional string incarnation_id = 2; } message IndexMetadataResponse { - string index_metadata_serialized_json = 1; + // TODO: Use `bytes` in conjunction with `Bytes`. + string index_metadata_json = 1; +} + +message ListIndexesRequest {} + +message ListIndexesResponse { + // TODO: Use `bytes` in conjunction with `Bytes`. + string indexes_metadata_json = 1; } -message ListAllSplitsRequest { +message DeleteIndexRequest { string index_uid = 1; } message ListSplitsRequest { - string filter_json = 1; + string index_uid = 1; + optional string list_splits_query_json = 2; } message ListSplitsResponse { - string splits_serialized_json = 1; + // TODO: Use `bytes` in conjunction with `Bytes`. + string splits_metadata_json = 1; } message StageSplitsRequest { string index_uid = 1; - string split_metadata_list_serialized_json = 2; + string splits_metadata_json = 2; } message PublishSplitsRequest { string index_uid = 1; - repeated string split_ids = 2; + repeated string staged_split_ids = 2; repeated string replaced_split_ids = 3; - optional string index_checkpoint_delta_serialized_json = 4; + optional SourceCheckpointDelta checkpoint_delta = 5; + + // Reserved fields due to deprecation + reserved 4; +} + +message SourceCheckpointDelta { + string source_id = 1; + map partition_deltas = 2; // A map from partition ID to partition delta. +} + +message PartitionDelta { + string from_position_exclusive = 1; + string to_position_inclusive = 2; } message MarkSplitsForDeletionRequest { @@ -145,11 +152,9 @@ message DeleteSplitsRequest { repeated string split_ids = 3; } -message SplitResponse {} - message AddSourceRequest { string index_uid = 1; - string source_config_serialized_json = 2; + string source_config_json = 2; } message ToggleSourceRequest { @@ -168,7 +173,6 @@ message ResetSourceCheckpointRequest { string source_id = 2; } -message SourceResponse {} /// /// Delete tasks. @@ -191,7 +195,7 @@ message DeleteQuery { // Query AST serialized in JSON string query_ast = 6; - // Reserved field due to deprecation + // Reserved fields due to deprecation reserved 4, 5; } @@ -201,8 +205,6 @@ message UpdateSplitsDeleteOpstampRequest { uint64 delete_opstamp = 3; } -message UpdateSplitsDeleteOpstampResponse {} - message LastDeleteOpstampRequest { string index_uid = 1; } @@ -211,12 +213,6 @@ message LastDeleteOpstampResponse { uint64 last_delete_opstamp = 1; } -message ListStaleSplitsRequest { - string index_uid = 1; - uint64 delete_opstamp = 2; - uint64 num_splits = 3; -} - message ListDeleteTasksRequest { string index_uid = 1; uint64 opstamp_start = 2; @@ -225,4 +221,3 @@ message ListDeleteTasksRequest { message ListDeleteTasksResponse { repeated DeleteTask delete_tasks = 1; } - diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index a5af0981ea6..26120712685 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -1,9 +1,15 @@ #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct EmptyResponse {} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct CreateIndexRequest { + #[prost(string, tag = "1")] + pub index_id: ::prost::alloc::string::String, #[prost(string, tag = "2")] - pub index_config_serialized_json: ::prost::alloc::string::String, + pub index_config_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -15,47 +21,36 @@ pub struct CreateIndexResponse { #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListIndexesMetadatasRequest {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListIndexesMetadatasResponse { +pub struct IndexMetadataRequest { #[prost(string, tag = "1")] - pub indexes_metadatas_serialized_json: ::prost::alloc::string::String, + pub index_id: ::prost::alloc::string::String, + #[prost(string, optional, tag = "2")] + pub incarnation_id: ::core::option::Option<::prost::alloc::string::String>, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct DeleteIndexRequest { +pub struct IndexMetadataResponse { + /// TODO: Use `bytes` in conjunction with `Bytes`. #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + pub index_metadata_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct DeleteTasksResponse {} +pub struct ListIndexesRequest {} #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct DeleteIndexResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct IndexMetadataRequest { +pub struct ListIndexesResponse { + /// TODO: Use `bytes` in conjunction with `Bytes`. #[prost(string, tag = "1")] - pub index_id: ::prost::alloc::string::String, + pub indexes_metadata_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct IndexMetadataResponse { - #[prost(string, tag = "1")] - pub index_metadata_serialized_json: ::prost::alloc::string::String, -} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListAllSplitsRequest { +pub struct DeleteIndexRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, } @@ -64,14 +59,17 @@ pub struct ListAllSplitsRequest { #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListSplitsRequest { #[prost(string, tag = "1")] - pub filter_json: ::prost::alloc::string::String, + pub index_uid: ::prost::alloc::string::String, + #[prost(string, optional, tag = "2")] + pub list_splits_query_json: ::core::option::Option<::prost::alloc::string::String>, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListSplitsResponse { + /// TODO: Use `bytes` in conjunction with `Bytes`. #[prost(string, tag = "1")] - pub splits_serialized_json: ::prost::alloc::string::String, + pub splits_metadata_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -80,7 +78,7 @@ pub struct StageSplitsRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] - pub split_metadata_list_serialized_json: ::prost::alloc::string::String, + pub splits_metadata_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -89,17 +87,37 @@ pub struct PublishSplitsRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, #[prost(string, repeated, tag = "2")] - pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + 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_serialized_json: ::core::option::Option< + #[prost(message, optional, tag = "5")] + pub checkpoint_delta: ::core::option::Option, +} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SourceCheckpointDelta { + #[prost(string, tag = "1")] + pub source_id: ::prost::alloc::string::String, + /// A map from partition ID to partition delta. + #[prost(map = "string, message", tag = "2")] + pub partition_deltas: ::std::collections::HashMap< ::prost::alloc::string::String, + PartitionDelta, >, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct PartitionDelta { + #[prost(string, tag = "1")] + pub from_position_exclusive: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub to_position_inclusive: ::prost::alloc::string::String, +} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct MarkSplitsForDeletionRequest { #[prost(string, tag = "2")] pub index_uid: ::prost::alloc::string::String, @@ -118,15 +136,11 @@ pub struct DeleteSplitsRequest { #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct SplitResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct AddSourceRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] - pub source_config_serialized_json: ::prost::alloc::string::String, + pub source_config_json: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -160,10 +174,6 @@ pub struct ResetSourceCheckpointRequest { #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct SourceResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteTask { #[prost(int64, tag = "1")] pub create_timestamp: i64, @@ -207,10 +217,6 @@ pub struct UpdateSplitsDeleteOpstampRequest { #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct UpdateSplitsDeleteOpstampResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct LastDeleteOpstampRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, @@ -225,17 +231,6 @@ pub struct LastDeleteOpstampResponse { #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListStaleSplitsRequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(uint64, tag = "2")] - pub delete_opstamp: u64, - #[prost(uint64, tag = "3")] - pub num_splits: u64, -} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct ListDeleteTasksRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, @@ -397,11 +392,11 @@ pub mod metastore_service_client { self.inner.unary(req, path, codec).await } /// Gets an indexes metadatas. - pub async fn list_indexes_metadatas( + pub async fn list_indexes( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -415,14 +410,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_indexes_metadatas", + "/quickwit.metastore.MetastoreService/list_indexes", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "list_indexes_metadatas", + "list_indexes", ), ); self.inner.unary(req, path, codec).await @@ -431,10 +426,7 @@ pub mod metastore_service_client { pub async fn delete_index( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -458,37 +450,6 @@ pub mod metastore_service_client { ); self.inner.unary(req, path, codec).await } - /// Gets all splits from index. - pub async fn list_all_splits( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_all_splits", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "list_all_splits", - ), - ); - self.inner.unary(req, path, codec).await - } /// Gets splits from index. pub async fn list_splits( &mut self, @@ -521,7 +482,7 @@ pub mod metastore_service_client { pub async fn stage_splits( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -549,7 +510,7 @@ pub mod metastore_service_client { pub async fn publish_splits( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -577,7 +538,7 @@ pub mod metastore_service_client { pub async fn mark_splits_for_deletion( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -605,7 +566,7 @@ pub mod metastore_service_client { pub async fn delete_splits( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -633,7 +594,7 @@ pub mod metastore_service_client { pub async fn add_source( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -658,7 +619,7 @@ pub mod metastore_service_client { pub async fn toggle_source( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -686,7 +647,7 @@ pub mod metastore_service_client { pub async fn delete_source( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -714,7 +675,7 @@ pub mod metastore_service_client { pub async fn reset_source_checkpoint( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result, tonic::Status> { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -801,10 +762,7 @@ pub mod metastore_service_client { pub async fn update_splits_delete_opstamp( &mut self, request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { + ) -> std::result::Result, tonic::Status> { self.inner .ready() .await @@ -859,37 +817,6 @@ pub mod metastore_service_client { ); self.inner.unary(req, path, codec).await } - /// / Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. - pub async fn list_stale_splits( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_stale_splits", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "list_stale_splits", - ), - ); - self.inner.unary(req, path, codec).await - } } } /// Generated server implementations. @@ -916,29 +843,18 @@ pub mod metastore_service_server { tonic::Status, >; /// Gets an indexes metadatas. - async fn list_indexes_metadatas( + async fn list_indexes( &self, - request: tonic::Request, + request: tonic::Request, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, >; /// Deletes an index async fn delete_index( &self, request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; - /// Gets all splits from index. - async fn list_all_splits( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; + ) -> std::result::Result, tonic::Status>; /// Gets splits from index. async fn list_splits( &self, @@ -951,42 +867,42 @@ pub mod metastore_service_server { async fn stage_splits( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Publishes split. async fn publish_splits( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Marks splits for deletion. async fn mark_splits_for_deletion( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Deletes splits. async fn delete_splits( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Adds source. async fn add_source( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Toggles source. async fn toggle_source( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Removes source. async fn delete_source( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Resets source checkpoint. async fn reset_source_checkpoint( &self, request: tonic::Request, - ) -> std::result::Result, tonic::Status>; + ) -> std::result::Result, tonic::Status>; /// Gets last opstamp for a given `index_id`. async fn last_delete_opstamp( &self, @@ -1004,10 +920,7 @@ pub mod metastore_service_server { async fn update_splits_delete_opstamp( &self, request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; + ) -> std::result::Result, tonic::Status>; /// Lists delete tasks with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. async fn list_delete_tasks( &self, @@ -1016,14 +929,6 @@ pub mod metastore_service_server { tonic::Response, tonic::Status, >; - /// / Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. - async fn list_stale_splits( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; } #[derive(Debug)] pub struct MetastoreServiceServer { @@ -1196,25 +1101,25 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_indexes_metadatas" => { + "/quickwit.metastore.MetastoreService/list_indexes" => { #[allow(non_camel_case_types)] - struct list_indexes_metadatasSvc(pub Arc); + struct list_indexesSvc(pub Arc); impl< T: MetastoreService, - > tonic::server::UnaryService - for list_indexes_metadatasSvc { - type Response = super::ListIndexesMetadatasResponse; + > tonic::server::UnaryService + for list_indexesSvc { + type Response = super::ListIndexesResponse; type Future = BoxFuture< tonic::Response, tonic::Status, >; fn call( &mut self, - request: tonic::Request, + request: tonic::Request, ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { - (*inner).list_indexes_metadatas(request).await + (*inner).list_indexes(request).await }; Box::pin(fut) } @@ -1226,7 +1131,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = list_indexes_metadatasSvc(inner); + let method = list_indexesSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1249,7 +1154,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for delete_indexSvc { - type Response = super::DeleteIndexResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1288,52 +1193,6 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_all_splits" => { - #[allow(non_camel_case_types)] - struct list_all_splitsSvc(pub Arc); - impl< - T: MetastoreService, - > tonic::server::UnaryService - for list_all_splitsSvc { - type Response = super::ListSplitsResponse; - 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 { - (*inner).list_all_splits(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 inner = inner.0; - let method = list_all_splitsSvc(inner); - let codec = tonic::codec::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/list_splits" => { #[allow(non_camel_case_types)] struct list_splitsSvc(pub Arc); @@ -1385,7 +1244,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for stage_splitsSvc { - type Response = super::SplitResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1431,7 +1290,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for publish_splitsSvc { - type Response = super::SplitResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1477,7 +1336,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for mark_splits_for_deletionSvc { - type Response = super::SplitResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1523,7 +1382,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for delete_splitsSvc { - type Response = super::SplitResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1569,7 +1428,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for add_sourceSvc { - type Response = super::SourceResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1613,7 +1472,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for toggle_sourceSvc { - type Response = super::SourceResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1659,7 +1518,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for delete_sourceSvc { - type Response = super::SourceResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1705,7 +1564,7 @@ pub mod metastore_service_server { T: MetastoreService, > tonic::server::UnaryService for reset_source_checkpointSvc { - type Response = super::SourceResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1846,7 +1705,7 @@ pub mod metastore_service_server { > tonic::server::UnaryService< super::UpdateSplitsDeleteOpstampRequest, > for update_splits_delete_opstampSvc { - type Response = super::UpdateSplitsDeleteOpstampResponse; + type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, @@ -1933,52 +1792,6 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_stale_splits" => { - #[allow(non_camel_case_types)] - struct list_stale_splitsSvc(pub Arc); - impl< - T: MetastoreService, - > tonic::server::UnaryService - for list_stale_splitsSvc { - type Response = super::ListSplitsResponse; - 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 { - (*inner).list_stale_splits(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 inner = inner.0; - let method = list_stale_splitsSvc(inner); - let codec = tonic::codec::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 { Ok( diff --git a/quickwit/quickwit-proto/src/control_plane/mod.rs b/quickwit/quickwit-proto/src/control_plane/mod.rs index 602ed62c6b6..5cb09df903c 100644 --- a/quickwit/quickwit-proto/src/control_plane/mod.rs +++ b/quickwit/quickwit-proto/src/control_plane/mod.rs @@ -23,6 +23,8 @@ use thiserror; #[path = "../codegen/quickwit/quickwit.control_plane.rs"] mod codegen; +pub use codegen::control_plane_service_grpc_client::ControlPlaneServiceGrpcClient; +pub use codegen::control_plane_service_grpc_server::ControlPlaneServiceGrpcServer; pub use codegen::*; pub type Result = std::result::Result; diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index 451d784f98a..4d88e59832a 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -28,6 +28,8 @@ use crate::{IndexUid, ServiceError, ServiceErrorCode}; #[path = "../codegen/quickwit/quickwit.indexing.rs"] mod codegen; +pub use codegen::indexing_service_grpc_client::IndexingServiceGrpcClient; +pub use codegen::indexing_service_grpc_server::IndexingServiceGrpcServer; pub use codegen::*; pub type Result = std::result::Result; diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index 6ecded6122a..4e6d2575e95 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -32,19 +32,20 @@ use tonic::service::Interceptor; use tonic::Status; use tracing::Span; use tracing_opentelemetry::OpenTelemetrySpanExt; -use ulid::Ulid; + +use crate::metastore::DeleteQuery; pub mod control_plane; pub mod indexing; -#[path = "codegen/quickwit/quickwit.metastore.rs"] pub mod metastore; #[path = "codegen/quickwit/quickwit.search.rs"] pub mod search; +pub mod types; -pub use metastore::*; +pub use search::sort_by_value::SortValue; pub use search::*; -pub use sort_by_value::SortValue; pub use tonic; +pub use types::{IndexUid, SourceId, SplitId}; pub mod jaeger { pub mod api_v2 { @@ -317,63 +318,6 @@ pub fn set_parent_span_from_request_metadata(request_metadata: &tonic::metadata: Span::current().set_parent(parent_cx); } -/// Index identifiers that uniquely identify not only the index, but also -/// its incarnation allowing to distinguish between deleted and recreated indexes. -/// It is represented as a stiring in index_id:incarnation_id format. -#[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Eq, Ord, PartialOrd, Hash)] -pub struct IndexUid(String); - -impl IndexUid { - /// Creates a new index uid form index_id and incarnation_id - pub fn new(index_id: impl Into) -> Self { - Self::from_parts(index_id, Ulid::new().to_string()) - } - - pub fn from_parts(index_id: impl Into, incarnation_id: impl Into) -> Self { - let incarnation_id = incarnation_id.into(); - let index_id = index_id.into(); - if incarnation_id.is_empty() { - Self(index_id) - } else { - Self(format!("{index_id}:{incarnation_id}")) - } - } - - pub fn index_id(&self) -> &str { - self.0.split(':').next().unwrap() - } - - pub fn incarnation_id(&self) -> &str { - if let Some(incarnation_id) = self.0.split(':').nth(1) { - incarnation_id - } else { - "" - } - } - - pub fn is_empty(&self) -> bool { - self.0.is_empty() - } -} - -impl From for String { - fn from(val: IndexUid) -> Self { - val.0 - } -} - -impl fmt::Display for IndexUid { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.0) - } -} - -impl From for IndexUid { - fn from(index_uid: String) -> Self { - IndexUid(index_uid) - } -} - // !!! Disclaimer !!! // // Prost imposes the PartialEq derived implementation. @@ -449,39 +393,3 @@ impl ServiceError for quickwit_actors::AskError } } } - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_index_uid_parsing() { - assert_eq!("foo", IndexUid::from("foo".to_string()).index_id()); - assert_eq!("foo", IndexUid::from("foo:bar".to_string()).index_id()); - assert_eq!("", IndexUid::from("foo".to_string()).incarnation_id()); - assert_eq!( - "bar", - IndexUid::from("foo:bar".to_string()).incarnation_id() - ); - } - - #[test] - fn test_index_uid_roundtrip() { - assert_eq!("foo", IndexUid::from("foo".to_string()).to_string()); - assert_eq!("foo:bar", IndexUid::from("foo:bar".to_string()).to_string()); - } - - #[test] - fn test_index_uid_roundtrip_using_parts() { - assert_eq!("foo", index_uid_roundtrip_using_parts("foo")); - assert_eq!("foo:bar", index_uid_roundtrip_using_parts("foo:bar")); - } - - fn index_uid_roundtrip_using_parts(index_uid: &str) -> String { - let index_uid = IndexUid::from(index_uid.to_string()); - let index_id = index_uid.index_id(); - let incarnation_id = index_uid.incarnation_id(); - let index_uid_from_parts = IndexUid::from_parts(index_id, incarnation_id); - index_uid_from_parts.to_string() - } -} diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs new file mode 100644 index 00000000000..7db7712a135 --- /dev/null +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -0,0 +1,156 @@ +// Copyright (C) 2023 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use crate::{IndexUid, SplitId}; + +#[path = "../codegen/quickwit/quickwit.metastore.rs"] +mod codegen; + +pub use codegen::metastore_service_client::MetastoreServiceClient; +pub use codegen::metastore_service_server::{MetastoreService, MetastoreServiceServer}; +pub use codegen::*; + +// Index API + +impl IndexMetadataRequest { + pub fn new(index_id: impl Into) -> Self { + Self { + index_id: index_id.into(), + incarnation_id: None, + } + } + + pub fn strict(index_uid: IndexUid) -> Self { + Self { + index_id: index_uid.index_id().to_string(), + incarnation_id: Some(index_uid.incarnation_id().to_string()), + } + } + + pub fn index_uid(&self) -> Option { + self.incarnation_id + .as_ref() + .map(|incarnation_id| IndexUid::from_parts(&self.index_id, incarnation_id)) + } +} + +impl DeleteIndexRequest { + pub fn new(index_uid: impl Into) -> Self { + Self { + index_uid: index_uid.into(), + } + } +} + +impl ListIndexesResponse { + pub fn num_indexes(&self) -> usize { + self.indexes_metadata_json.len() + } +} + +// Split API + +impl ListSplitsRequest { + pub fn all(index_uid: impl Into) -> Self { + Self { + index_uid: index_uid.into().into(), + list_splits_query_json: None, + } + } +} + +impl PublishSplitsRequest { + pub fn new( + index_uid: impl Into, + staged_split_ids: impl IntoIterator>, + replaced_split_ids: impl IntoIterator>, + checkpoint_delta: Option, + ) -> Self { + Self { + index_uid: index_uid.into(), + staged_split_ids: staged_split_ids + .into_iter() + .map(|split_id| split_id.into()) + .collect(), + replaced_split_ids: replaced_split_ids + .into_iter() + .map(|split_id| split_id.into()) + .collect(), + checkpoint_delta, + } + } +} + +impl MarkSplitsForDeletionRequest { + pub fn new( + index_uid: impl Into, + split_ids: impl IntoIterator>, + ) -> Self { + Self { + index_uid: index_uid.into(), + split_ids: split_ids + .into_iter() + .map(|split_id| split_id.into()) + .collect(), + } + } +} + +impl DeleteSplitsRequest { + pub fn new( + index_uid: impl Into, + split_ids: impl IntoIterator>, + ) -> Self { + Self { + index_uid: index_uid.into(), + split_ids: split_ids + .into_iter() + .map(|split_id| split_id.into()) + .collect(), + } + } +} + +// Delete task API + +impl ListDeleteTasksRequest { + pub fn new(index_uid: impl Into, opstamp_start: u64) -> Self { + Self { + index_uid: index_uid.into(), + opstamp_start, + } + } +} + +impl UpdateSplitsDeleteOpstampRequest { + pub fn new( + index_uid: impl Into, + split_ids: impl IntoIterator>, + delete_opstamp: u64, + ) -> Self { + Self { + index_uid: index_uid.into(), + split_ids: split_ids + .into_iter() + .map(|split_id| split_id.into()) + .collect(), + delete_opstamp, + } + } +} diff --git a/quickwit/quickwit-proto/src/types.rs b/quickwit/quickwit-proto/src/types.rs new file mode 100644 index 00000000000..eb667af0f59 --- /dev/null +++ b/quickwit/quickwit-proto/src/types.rs @@ -0,0 +1,132 @@ +// Copyright (C) 2023 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::fmt; + +use serde::{Deserialize, Serialize}; +use ulid::Ulid; + +pub type SourceId = String; +pub type SplitId = String; + +/// Index identifiers that uniquely identify not only the index, but also +/// its incarnation allowing to distinguish between deleted and recreated indexes. +/// It is represented as a stiring in index_id:incarnation_id format. +#[derive(Clone, Default, PartialEq, Eq, Ord, PartialOrd, Hash, Serialize, Deserialize)] +pub struct IndexUid(String); + +impl IndexUid { + /// Creates a new index UID from an `index_id` and `incarnation_id`. + pub fn new(index_id: impl Into) -> Self { + Self::from_parts(index_id, Ulid::new().to_string()) + } + + pub fn from_parts(index_id: impl Into, incarnation_id: impl Into) -> Self { + let incarnation_id = incarnation_id.into(); + let index_id = index_id.into(); + if incarnation_id.is_empty() { + Self(index_id) + } else { + Self(format!("{index_id}:{incarnation_id}")) + } + } + + pub fn index_id(&self) -> &str { + self.0.split(':').next().unwrap() + } + + pub fn incarnation_id(&self) -> &str { + if let Some(incarnation_id) = self.0.split(':').nth(1) { + incarnation_id + } else { + "" + } + } + + pub fn as_str(&self) -> &str { + &self.0 + } + + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } +} + +impl fmt::Display for IndexUid { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + +impl fmt::Debug for IndexUid { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("IndexUid") + .field("index_id", &self.index_id()) + .field("incarnation_id", &self.incarnation_id()) + .finish() + } +} + +impl From for String { + fn from(val: IndexUid) -> Self { + val.0 + } +} + +impl From for IndexUid { + fn from(index_uid: String) -> Self { + IndexUid(index_uid) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_index_uid_parsing() { + assert_eq!("foo", IndexUid::from("foo".to_string()).index_id()); + assert_eq!("foo", IndexUid::from("foo:bar".to_string()).index_id()); + assert_eq!("", IndexUid::from("foo".to_string()).incarnation_id()); + assert_eq!( + "bar", + IndexUid::from("foo:bar".to_string()).incarnation_id() + ); + } + + #[test] + fn test_index_uid_roundtrip() { + assert_eq!("foo", IndexUid::from("foo".to_string()).to_string()); + assert_eq!("foo:bar", IndexUid::from("foo:bar".to_string()).to_string()); + } + + #[test] + fn test_index_uid_roundtrip_using_parts() { + assert_eq!("foo", index_uid_roundtrip_using_parts("foo")); + assert_eq!("foo:bar", index_uid_roundtrip_using_parts("foo:bar")); + } + + fn index_uid_roundtrip_using_parts(index_uid: &str) -> String { + let index_uid = IndexUid::from(index_uid.to_string()); + let index_id = index_uid.index_id(); + let incarnation_id = index_uid.incarnation_id(); + let index_uid_from_parts = IndexUid::from_parts(index_id, incarnation_id); + index_uid_from_parts.to_string() + } +} diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 33d03fdaaa7..8cc5b42625c 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -60,7 +60,11 @@ use std::sync::Arc; pub use find_trace_ids_collector::FindTraceIdsCollector; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitMetadata, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, Metastore, SplitMetadata, + SplitState, +}; +use quickwit_proto::metastore::ListSplitsRequest; use quickwit_proto::{IndexUid, PartialHit, SearchRequest, SplitIdAndFooterOffsets}; use quickwit_storage::StorageResolver; use tantivy::DocAddress; @@ -124,16 +128,14 @@ async fn list_relevant_splits( search_request: &SearchRequest, metastore: &dyn Metastore, ) -> crate::Result> { - let mut query = ListSplitsQuery::for_index(index_uid).with_split_state(SplitState::Published); + let mut list_splits_query = ListSplitsQuery::default().with_split_state(SplitState::Published); if let Some(start_ts) = search_request.start_timestamp { - query = query.with_time_range_start_gte(start_ts); + list_splits_query = list_splits_query.with_time_range_start_gte(start_ts); } - if let Some(end_ts) = search_request.end_timestamp { - query = query.with_time_range_end_lt(end_ts); + list_splits_query = list_splits_query.with_time_range_end_lt(end_ts); } - let query_ast: QueryAst = serde_json::from_str(&search_request.query_ast).map_err(|_| { SearchError::InternalError(format!( "Failed to deserialize query_ast: `{}`", @@ -141,14 +143,13 @@ async fn list_relevant_splits( )) })?; if let Some(tags_filter) = extract_tags_from_query(query_ast) { - query = query.with_tags_filter(tags_filter); + list_splits_query = list_splits_query.with_tags_filter(tags_filter); } - - let split_metas = metastore.list_splits(query).await?; - Ok(split_metas - .into_iter() - .map(|metadata| metadata.split_metadata) - .collect::>()) + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid, list_splits_query)?; + let list_splits_response = metastore.list_splits(list_splits_request).await?; + let splits_metadata = list_splits_response.deserialize_splits_metadata()?; + Ok(splits_metadata) } /// Converts a Tantivy `NamedFieldDocument` into a json string using the diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index e875cec2237..419ef2190c2 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -24,7 +24,11 @@ use futures::future::try_join_all; use itertools::Itertools; use quickwit_config::{build_doc_mapper, IndexConfig}; use quickwit_doc_mapper::{DocMapper, DYNAMIC_FIELD_NAME}; -use quickwit_metastore::{Metastore, SplitMetadata}; +use quickwit_metastore::{ + IndexMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, + Metastore, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::{IndexMetadataRequest, ListSplitsRequest}; use quickwit_proto::{ FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, ListTermsRequest, ListTermsResponse, PartialHit, @@ -235,7 +239,9 @@ pub async fn root_search( ) -> crate::Result { let start_instant = tokio::time::Instant::now(); - let index_metadata = metastore.index_metadata(&search_request.index_id).await?; + let index_metadata_request = IndexMetadataRequest::new(&search_request.index_id); + let index_metadata_response = metastore.index_metadata(index_metadata_request).await?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); @@ -270,10 +276,10 @@ pub async fn root_search( SearchError::InternalError(format!("Failed to serialize doc mapper: Cause {err}")) })?; - let split_metadatas: Vec = + let splits_metadata: Vec = list_relevant_splits(index_uid, &search_request, metastore).await?; - let split_offsets_map: HashMap = split_metadatas + let split_offsets_map: HashMap = splits_metadata .iter() .map(|metadata| { ( @@ -285,7 +291,7 @@ pub async fn root_search( let index_uri = &index_config.index_uri; - let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); + let jobs: Vec = splits_metadata.iter().map(SearchJob::from).collect(); let assigned_leaf_search_jobs = cluster_client .search_job_placer @@ -610,9 +616,9 @@ pub async fn root_list_terms( ) -> crate::Result { let start_instant = tokio::time::Instant::now(); - let index_metadata = metastore - .index_metadata(&list_terms_request.index_id) - .await?; + let index_metadata_request = IndexMetadataRequest::new(&list_terms_request.index_id); + let index_metadata_response = metastore.index_metadata(index_metadata_request).await?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config: IndexConfig = index_metadata.into_index_config(); @@ -635,28 +641,22 @@ pub async fn root_list_terms( "Trying to list terms on field which isn't indexed".to_string(), )); } - - let mut query = quickwit_metastore::ListSplitsQuery::for_index(index_uid) - .with_split_state(quickwit_metastore::SplitState::Published); + let mut query = ListSplitsQuery::default().with_split_state(SplitState::Published); if let Some(start_ts) = list_terms_request.start_timestamp { query = query.with_time_range_start_gte(start_ts); } - if let Some(end_ts) = list_terms_request.end_timestamp { query = query.with_time_range_end_lt(end_ts); } - - let split_metadatas = metastore - .list_splits(query) - .await? - .into_iter() - .map(|metadata| metadata.split_metadata) - .collect::>(); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(index_uid.clone(), query)?; + let list_splits_response = metastore.list_splits(list_splits_request).await?; + let splits_metadata = list_splits_response.deserialize_splits_metadata()?; let index_uri = &index_config.index_uri; - let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); + let jobs: Vec = splits_metadata.iter().map(SearchJob::from).collect(); let assigned_leaf_search_jobs = cluster_client .search_job_placer .assign_jobs(jobs, &HashSet::default()) diff --git a/quickwit/quickwit-search/src/search_stream/root.rs b/quickwit/quickwit-search/src/search_stream/root.rs index 64ebef9ad0f..84efe1a2a7f 100644 --- a/quickwit/quickwit-search/src/search_stream/root.rs +++ b/quickwit/quickwit-search/src/search_stream/root.rs @@ -23,7 +23,8 @@ use bytes::Bytes; use futures::{StreamExt, TryStreamExt}; use quickwit_common::uri::Uri; use quickwit_config::build_doc_mapper; -use quickwit_metastore::Metastore; +use quickwit_metastore::{IndexMetadataResponseExt, Metastore}; +use quickwit_proto::metastore::IndexMetadataRequest; use quickwit_proto::{LeafSearchStreamRequest, SearchRequest, SearchStreamRequest}; use quickwit_query::query_ast::QueryAst; use tokio_stream::StreamMap; @@ -43,19 +44,19 @@ pub async fn root_search_stream( // TODO: building a search request should not be necessary for listing splits. // This needs some refactoring: relevant splits, metadata_map, jobs... - let index_metadata = metastore - .index_metadata(&search_stream_request.index_id) - .await?; + let index_metadata_request = IndexMetadataRequest::new(&search_stream_request.index_id); + let index_metadata_response = metastore.index_metadata(index_metadata_request).await?; + let index_metadata = index_metadata_response.deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); let doc_mapper = build_doc_mapper(&index_config.doc_mapping, &index_config.search_settings) - .map_err(|err| { - SearchError::InternalError(format!("Failed to build doc mapper. Cause: {err}")) + .map_err(|error| { + SearchError::InternalError(format!("Failed to build doc mapper. Cause: {error}")) })?; let query_ast: QueryAst = serde_json::from_str(&search_stream_request.query_ast) - .map_err(|err| SearchError::InvalidQuery(err.to_string()))?; + .map_err(|error| SearchError::InvalidQuery(error.to_string()))?; let query_ast_resolved = query_ast.parse_user_query(doc_mapper.default_search_fields())?; if let Some(timestamp_field) = doc_mapper.timestamp_field_name() { @@ -66,20 +67,19 @@ pub async fn root_search_stream( &mut search_stream_request.end_timestamp, ); } - // Validates the query by effectively building it against the current schema. doc_mapper.query(doc_mapper.schema(), &query_ast_resolved, true)?; search_stream_request.query_ast = serde_json::to_string(&query_ast_resolved)?; let search_request = SearchRequest::try_from(search_stream_request.clone())?; - let split_metadatas = list_relevant_splits(index_uid, &search_request, metastore).await?; + let splits_metadata = list_relevant_splits(index_uid, &search_request, metastore).await?; - let doc_mapper_str = serde_json::to_string(&doc_mapper).map_err(|err| { - SearchError::InternalError(format!("Failed to serialize doc mapper: Cause {err}")) + let doc_mapper_str = serde_json::to_string(&doc_mapper).map_err(|error| { + SearchError::InternalError(format!("Failed to serialize doc mapper: Cause {error}")) })?; let index_uri: &Uri = &index_config.index_uri; - let leaf_search_jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); + let leaf_search_jobs: Vec = splits_metadata.iter().map(SearchJob::from).collect(); let assigned_leaf_search_jobs = cluster_client .search_job_placer .assign_jobs(leaf_search_jobs, &HashSet::default())