diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 3a837986fd..9ea39cc9fa 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -5758,6 +5758,7 @@ dependencies = [ "md5", "mockall", "once_cell", + "ouroboros", "quickwit-common", "quickwit-config", "quickwit-doc-mapper", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 9156b457d4..319eb46dda 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -122,6 +122,7 @@ openssl = { version = "0.10.60", default-features = false } openssl-probe = "0.1.5" opentelemetry = { version = "0.19", features = ["rt-tokio"] } opentelemetry-otlp = "0.12.0" +ouroboros = "0.18.0" pin-project = "1.1.0" pnet = { version = "0.33.0", features = ["std"] } postcard = { version = "1.0.4", features = [ diff --git a/quickwit/quickwit-cli/tests/cli.rs b/quickwit/quickwit-cli/tests/cli.rs index 4445b2cc0d..f4a9491d5d 100644 --- a/quickwit/quickwit-cli/tests/cli.rs +++ b/quickwit/quickwit-cli/tests/cli.rs @@ -42,8 +42,8 @@ use quickwit_common::rand::append_random_suffix; use quickwit_common::uri::Uri; use quickwit_config::{SourceInputFormat, CLI_INGEST_SOURCE_ID}; use quickwit_metastore::{ - ListSplitsRequestExt, ListSplitsResponseExt, MetastoreResolver, MetastoreServiceExt, - SplitState, StageSplitsRequestExt, + ListSplitsRequestExt, MetastoreResolver, MetastoreServiceExt, MetastoreServiceStreamSplitsExt, + SplitMetadata, SplitState, StageSplitsRequestExt, }; use quickwit_proto::metastore::{ DeleteSplitsRequest, EntityKind, IndexMetadataRequest, ListSplitsRequest, @@ -249,17 +249,18 @@ async fn test_ingest_docs_cli() { local_ingest_docs_cli(args).await.unwrap(); - let splits: Vec<_> = test_env + let splits_metadata: Vec = test_env .metastore() .await .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap(); - assert_eq!(splits.len(), 1); - assert_eq!(splits[0].split_metadata.num_docs, 5); + assert_eq!(splits_metadata.len(), 1); + assert_eq!(splits_metadata[0].num_docs, 5); // Ensure cache directory is empty. let cache_directory_path = get_cache_directory_path(&test_env.data_dir_path); @@ -665,13 +666,14 @@ async fn test_garbage_collect_cli_no_grace() { dry_run, }; - let splits = metastore + let splits_metadata = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap(); - assert_eq!(splits.len(), 1); + assert_eq!(splits_metadata.len(), 1); let args = create_gc_args(false); @@ -681,7 +683,7 @@ async fn test_garbage_collect_cli_no_grace() { let index_path = test_env.indexes_dir_path.join(&test_env.index_id); assert_eq!(index_path.try_exists().unwrap(), true); - let split_ids = vec![splits[0].split_id().to_string()]; + let split_ids = vec![splits_metadata[0].split_id().to_string()]; let mut metastore = refresh_metastore(metastore).await.unwrap(); let mark_for_deletion_request = MarkSplitsForDeletionRequest::new(index_uid.clone(), split_ids.clone()); @@ -718,7 +720,8 @@ async fn test_garbage_collect_cli_no_grace() { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap() .len(), 0 @@ -776,16 +779,17 @@ async fn test_garbage_collect_index_cli() { .await .unwrap(); - let splits = metastore + let splits_metadata = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap(); - assert_eq!(splits.len(), 1); + assert_eq!(splits_metadata.len(), 1); let index_path = test_env.indexes_dir_path.join(&test_env.index_id); - let split_filename = quickwit_common::split_file(splits[0].split_metadata.split_id.as_str()); + let split_filename = quickwit_common::split_file(splits_metadata[0].split_id.as_str()); let split_path = index_path.join(&split_filename); assert_eq!(split_path.try_exists().unwrap(), true); @@ -795,41 +799,39 @@ async fn test_garbage_collect_index_cli() { // Split should still exists within grace period. let mut metastore = refresh_metastore(metastore).await.unwrap(); - let splits = metastore + let splits_metadata = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap(); - assert_eq!(splits.len(), 1); + assert_eq!(splits_metadata.len(), 1); // The following steps help turn an existing published split into a staged one // without deleting the files. - let split = splits[0].clone(); + let split_metadata = splits_metadata[0].clone(); metastore .mark_splits_for_deletion(MarkSplitsForDeletionRequest::new( index_uid.clone(), - vec![split.split_metadata.split_id.to_string()], + vec![split_metadata.split_id.to_string()], )) .await .unwrap(); metastore .delete_splits(DeleteSplitsRequest { index_uid: index_uid.to_string(), - split_ids: splits + split_ids: splits_metadata .into_iter() - .map(|split| split.split_metadata.split_id) + .map(|split_metadata| split_metadata.split_id) .collect(), }) .await .unwrap(); metastore .stage_splits( - StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split.split_metadata.clone(), - ) - .unwrap(), + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata.clone()) + .unwrap(), ) .await .unwrap(); @@ -840,7 +842,8 @@ async fn test_garbage_collect_index_cli() { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits[0].split_state, SplitState::Staged); @@ -855,7 +858,8 @@ async fn test_garbage_collect_index_cli() { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits.len(), 1); assert_eq!(splits[0].split_state, SplitState::Staged); @@ -873,7 +877,8 @@ async fn test_garbage_collect_index_cli() { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); // Splits should be deleted from both metastore and file system. assert_eq!(splits.len(), 0); diff --git a/quickwit/quickwit-common/src/stream_utils.rs b/quickwit/quickwit-common/src/stream_utils.rs index c3c29b1381..e2d54b70e8 100644 --- a/quickwit/quickwit-common/src/stream_utils.rs +++ b/quickwit/quickwit-common/src/stream_utils.rs @@ -33,6 +33,20 @@ pub struct ServiceStream { inner: BoxStream, } +impl ServiceStream +where T: Send + 'static +{ + pub fn new(inner: BoxStream) -> Self { + Self { inner } + } + + pub fn empty() -> Self { + Self { + inner: Box::pin(stream::empty()), + } + } +} + impl fmt::Debug for ServiceStream where T: 'static { @@ -159,3 +173,14 @@ where T: Send + 'static } } } + +#[cfg(any(test, feature = "testsuite"))] +impl From> for ServiceStream +where T: Send + 'static +{ + fn from(values: Vec) -> Self { + Self { + inner: Box::pin(stream::iter(values)), + } + } +} diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index 9246fda84f..4c6a8743ee 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -23,14 +23,14 @@ use std::sync::Arc; use std::time::Duration; use futures::Future; -use quickwit_common::{PrettySample, Progress}; +use quickwit_common::{PrettySample, Progress, ServiceStream}; use quickwit_metastore::{ - ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitInfo, SplitMetadata, - SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitInfo, + SplitMetadata, SplitState, }; use quickwit_proto::metastore::{ - DeleteSplitsRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, - MetastoreService, MetastoreServiceClient, + DeleteSplitsRequest, ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, + MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, }; use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_storage::{BulkDeleteError, Storage}; @@ -106,7 +106,8 @@ pub async fn run_garbage_collect( metastore.list_splits(list_deletable_staged_request), ) .await? - .deserialize_splits_metadata()?; + .collect_splits_metadata() + .await?; if dry_run { let marked_for_deletion_query = ListSplitsQuery::for_index(index_uid.clone()) @@ -118,7 +119,8 @@ pub async fn run_garbage_collect( metastore.list_splits(marked_for_deletion_request), ) .await? - .deserialize_splits_metadata()?; + .collect_splits_metadata() + .await?; splits_marked_for_deletion.extend(deletable_staged_splits); let candidate_entries: Vec = splits_marked_for_deletion @@ -191,23 +193,27 @@ async fn delete_splits_marked_for_deletion( break; } }; - let list_splits_result = - protect_future(progress_opt, metastore.list_splits(list_splits_request)) - .await - .and_then(|list_splits_response| list_splits_response.deserialize_splits()); + let splits_stream_result = + protect_future(progress_opt, metastore.list_splits(list_splits_request)).await; + let splits_to_delete_stream: ServiceStream> = + match splits_stream_result { + Ok(splits_stream) => splits_stream, + Err(error) => { + error!(error = ?error, "failed to fetch stream splits"); + break; + } + }; - let splits_to_delete: Vec = match list_splits_result { - Ok(splits) => splits - .into_iter() - .map(|split| split.split_metadata) - .collect(), - Err(error) => { - error!(error = ?error, "failed to fetch deletable splits"); - break; - } - }; + let splits_metadata_to_delete: Vec = + match splits_to_delete_stream.collect_splits_metadata().await { + Ok(splits) => splits, + Err(error) => { + error!(error = ?error, "failed to collect splits"); + break; + } + }; - let num_splits_to_delete = splits_to_delete.len(); + let num_splits_to_delete = splits_metadata_to_delete.len(); if num_splits_to_delete == 0 { break; @@ -216,7 +222,7 @@ async fn delete_splits_marked_for_deletion( index_uid.clone(), storage.clone(), metastore.clone(), - splits_to_delete, + splits_metadata_to_delete, progress_opt, ) .await; @@ -345,14 +351,13 @@ mod tests { use std::time::Duration; use itertools::Itertools; + use quickwit_common::ServiceStream; use quickwit_config::IndexConfig; use quickwit_metastore::{ - metastore_for_test, CreateIndexRequestExt, ListSplitsQuery, SplitMetadata, SplitState, - StageSplitsRequestExt, - }; - use quickwit_proto::metastore::{ - CreateIndexRequest, EntityKind, ListSplitsResponse, StageSplitsRequest, + metastore_for_test, CreateIndexRequestExt, ListSplitsQuery, + MetastoreServiceStreamSplitsExt, SplitMetadata, SplitState, StageSplitsRequestExt, }; + use quickwit_proto::metastore::{CreateIndexRequest, EntityKind, StageSplitsRequest}; use quickwit_proto::types::IndexUid; use quickwit_storage::{ storage_for_test, BulkDeleteError, DeleteFailure, MockStorage, PutPayload, @@ -395,7 +400,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 1 @@ -422,7 +428,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 1 @@ -449,7 +456,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 1 @@ -496,7 +504,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 1 @@ -523,7 +532,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 1 @@ -549,7 +559,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .len(), 0 @@ -564,7 +575,7 @@ mod tests { metastore .expect_list_splits() .times(2) - .returning(|_| Ok(ListSplitsResponse::empty())); + .returning(|_| Ok(ServiceStream::empty())); run_garbage_collect( IndexUid::new_with_random_ulid("index-test-gc-deletes"), storage.clone(), @@ -621,7 +632,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits.len(), 1); @@ -646,7 +658,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .is_empty()); } @@ -739,7 +752,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits.len(), 1); assert_eq!(splits[0].split_id(), split_id_1); diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 5a5fbec3b9..e40118c6c5 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -25,8 +25,8 @@ use quickwit_config::{validate_identifier, IndexConfig, SourceConfig}; use quickwit_indexing::check_source_connectivity; use quickwit_metastore::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, - ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitInfo, SplitMetadata, - SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitInfo, + SplitMetadata, SplitState, }; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, EntityKind, IndexMetadataRequest, @@ -174,25 +174,27 @@ impl IndexService { if dry_run { let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid)?; - let splits_to_delete = self + let splits_to_delete: Vec = self .metastore .list_splits(list_splits_request) .await? - .deserialize_splits()? + .collect_splits() + .await? .into_iter() .map(|split| split.split_metadata.as_split_info()) - .collect::>(); + .collect(); return Ok(splits_to_delete); } // Schedule staged and published splits for deletion. let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_states([SplitState::Staged, SplitState::Published]); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; - let split_ids = self + let split_ids: Vec = self .metastore .list_splits(list_splits_request) .await? - .deserialize_split_ids()?; + .collect_split_ids() + .await?; let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new(index_uid.clone(), split_ids); self.metastore @@ -203,17 +205,18 @@ impl IndexService { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_state(SplitState::MarkedForDeletion); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; - let splits_to_delete = self + let splits_metadata_to_delete: Vec = self .metastore .list_splits(list_splits_request) .await? - .deserialize_splits_metadata()?; + .collect_splits_metadata() + .await?; let deleted_splits = delete_splits_from_storage_and_metastore( index_uid.clone(), storage, self.metastore.clone(), - splits_to_delete, + splits_metadata_to_delete, None, ) .await?; @@ -291,7 +294,8 @@ impl IndexService { .metastore .list_splits(list_splits_request) .await? - .deserialize_splits_metadata()?; + .collect_splits_metadata() + .await?; let split_ids: Vec = splits_metadata .iter() .map(|split| split.split_id.to_string()) @@ -417,7 +421,9 @@ mod tests { use quickwit_common::uri::Uri; use quickwit_config::IndexConfig; - use quickwit_metastore::{metastore_for_test, SplitMetadata, StageSplitsRequestExt}; + use quickwit_metastore::{ + metastore_for_test, MetastoreServiceExt, SplitMetadata, StageSplitsRequestExt, + }; use quickwit_proto::metastore::StageSplitsRequest; use quickwit_storage::PutPayload; @@ -497,7 +503,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits.len(), 1); @@ -510,13 +517,15 @@ mod tests { let split_infos = index_service.delete_index(index_id, false).await.unwrap(); assert_eq!(split_infos.len(), 1); - let error = metastore + assert!(!metastore.index_exists(index_id).await.unwrap()); + let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await - .unwrap_err(); - assert!( - matches!(error, MetastoreError::NotFound(EntityKind::Index { index_id }) if index_id == index_uid.index_id()) - ); + .unwrap() + .collect_splits() + .await + .unwrap(); + assert!(splits.is_empty()); assert!(!storage.exists(split_path).await.unwrap()); } } diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 6231b5f164..b594e6ced3 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -612,13 +612,13 @@ mod tests { use std::sync::Arc; use quickwit_actors::{Command, Universe}; + use quickwit_common::ServiceStream; use quickwit_config::{IndexingSettings, SourceInputFormat, SourceParams, VoidSourceParams}; use quickwit_doc_mapper::{default_doc_mapper_for_test, DefaultDocMapper}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; - use quickwit_metastore::{IndexMetadata, ListSplitsResponseExt, PublishSplitsRequestExt}; + use quickwit_metastore::{IndexMetadata, PublishSplitsRequestExt}; use quickwit_proto::metastore::{ - EmptyResponse, IndexMetadataResponse, LastDeleteOpstampResponse, ListSplitsResponse, - MetastoreError, + EmptyResponse, IndexMetadataResponse, LastDeleteOpstampResponse, MetastoreError, }; use quickwit_proto::types::{IndexUid, PipelineUid}; use quickwit_storage::RamStorage; @@ -851,7 +851,7 @@ mod tests { }); mock_metastore .expect_list_splits() - .returning(|_| Ok(ListSplitsResponse::empty())); + .returning(|_| Ok(ServiceStream::empty())); let universe = Universe::with_accelerated_time(); let node_id = "test-node"; let doc_mapper = Arc::new(default_doc_mapper_for_test()); diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 94ad7c382a..5b625ed8dc 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -860,6 +860,7 @@ mod tests { use quickwit_actors::{Health, ObservationType, Supervisable, Universe, HEARTBEAT}; use quickwit_cluster::{create_cluster_for_test, ChannelTransport}; use quickwit_common::rand::append_random_suffix; + use quickwit_common::ServiceStream; use quickwit_config::{ IngestApiConfig, KafkaSourceParams, SourceConfig, SourceInputFormat, SourceParams, VecSourceParams, @@ -867,12 +868,12 @@ mod tests { use quickwit_ingest::{init_ingest_api, CreateQueueIfNotExistsRequest}; use quickwit_metastore::{ metastore_for_test, AddSourceRequestExt, CreateIndexRequestExt, - ListIndexesMetadataResponseExt, ListSplitsResponseExt, + ListIndexesMetadataResponseExt, }; use quickwit_proto::indexing::IndexingTask; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, IndexMetadataResponse, - ListIndexesMetadataResponse, ListSplitsResponse, + ListIndexesMetadataResponse, }; use super::*; @@ -1467,7 +1468,7 @@ mod tests { }); metastore .expect_list_splits() - .returning(|_| Ok(ListSplitsResponse::try_from_splits(Vec::new()).unwrap())); + .returning(|_| Ok(ServiceStream::empty())); let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); let (indexing_service, indexing_service_handle) = spawn_indexing_service_for_test( diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index 81f4f642f1..04a54f3d5f 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -560,7 +560,7 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::split_file; use quickwit_metastore::{ - ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, StageSplitsRequestExt, + ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitMetadata, StageSplitsRequestExt, }; use quickwit_proto::metastore::{ DeleteQuery, ListSplitsRequest, PublishSplitsRequest, StageSplitsRequest, @@ -606,7 +606,8 @@ mod tests { .list_splits(list_splits_request) .await .unwrap() - .deserialize_splits_metadata() + .collect_splits_metadata() + .await .unwrap(); assert_eq!(split_metas.len(), 4); let merge_scratch_directory = TempDirectory::for_test(); @@ -729,17 +730,16 @@ mod tests { query_ast: quickwit_query::query_ast::qast_json_helper(delete_query, &["body"]), }) .await?; - let split = metastore + let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits() - .unwrap() - .into_iter() - .next() + .collect_splits() + .await .unwrap(); + // We want to test a delete on a split with num_merge_ops > 0. - let mut new_split_metadata = split.split_metadata.clone(); + let mut new_split_metadata = splits[0].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( @@ -751,7 +751,7 @@ mod tests { let publish_splits_request = PublishSplitsRequest { index_uid: index_uid.to_string(), staged_split_ids: vec![new_split_metadata.split_id.to_string()], - replaced_split_ids: vec![split.split_metadata.split_id.to_string()], + replaced_split_ids: vec![splits[0].split_metadata.split_id.to_string()], index_checkpoint_delta_json_opt: None, publish_token_opt: None, }; @@ -764,7 +764,7 @@ mod tests { let merge_scratch_directory = TempDirectory::for_test(); let downloaded_splits_directory = merge_scratch_directory.named_temp_child("downloaded-splits-")?; - let split_filename = split_file(split.split_metadata.split_id()); + let split_filename = split_file(splits[0].split_metadata.split_id()); let new_split_filename = split_file(new_split_metadata.split_id()); let dest_filepath = downloaded_splits_directory.path().join(&new_split_filename); test_sandbox @@ -846,7 +846,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap() .into_iter() .all( diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index c4509e08a2..42ee16976c 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -32,7 +32,7 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_common::KillSwitch; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::{ - ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitState, }; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{ @@ -223,10 +223,12 @@ impl MergePipeline { .with_split_state(SplitState::Published) .retain_immature(OffsetDateTime::now_utc()); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; - let published_splits_metadata: Vec = ctx + let published_splits_stream = ctx .protect_future(self.params.metastore.list_splits(list_splits_request)) - .await? - .deserialize_splits_metadata()?; + .await?; + let published_splits_metadata = ctx + .protect_future(published_splits_stream.collect_splits_metadata()) + .await?; info!( num_splits = published_splits_metadata.len(), @@ -485,10 +487,11 @@ mod tests { use quickwit_actors::{ActorExitStatus, Universe}; use quickwit_common::temp_dir::TempDirectory; + use quickwit_common::ServiceStream; use quickwit_doc_mapper::default_doc_mapper_for_test; - use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; + use quickwit_metastore::ListSplitsRequestExt; use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::metastore::{ListSplitsResponse, MetastoreServiceClient}; + use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::types::{IndexUid, PipelineUid}; use quickwit_storage::RamStorage; @@ -521,7 +524,7 @@ mod tests { }; true }) - .returning(|_| Ok(ListSplitsResponse::try_from_splits(Vec::new()).unwrap())); + .returning(|_| Ok(ServiceStream::empty())); let universe = Universe::with_accelerated_time(); let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage.clone()); diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 5308a547ed..b1e5a8ec81 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -281,7 +281,7 @@ pub fn mock_split_meta(split_id: &str, index_uid: &IndexUid) -> SplitMetadata { #[cfg(test)] mod tests { - use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; + use quickwit_metastore::{ListSplitsRequestExt, MetastoreServiceStreamSplitsExt}; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; use super::TestSandbox; @@ -312,7 +312,8 @@ mod tests { ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap(), ) .await? - .deserialize_splits()?; + .collect_splits() + .await?; assert_eq!(splits.len(), 1); test_sandbox.add_documents(vec![ serde_json::json!({"title": "Byzantine-Ottoman wars", "body": "...", "url": "http://biz-ottoman"}), @@ -324,7 +325,8 @@ mod tests { ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap(), ) .await? - .deserialize_splits()?; + .collect_splits() + .await?; assert_eq!(splits.len(), 2); } test_sandbox.assert_quit().await; diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 7a4c5dbbd1..3650718dda 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -291,7 +291,7 @@ mod tests { use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_indexing::TestSandbox; - use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt, SplitState}; + use quickwit_metastore::{ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitState}; use quickwit_proto::metastore::{DeleteQuery, ListSplitsRequest, MetastoreService}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{ @@ -420,7 +420,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); assert_eq!(splits.len(), 2); let published_split = splits diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index 75d40b59ad..0c6b519ada 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -425,7 +425,10 @@ mod tests { use quickwit_config::build_doc_mapper; use quickwit_indexing::merge_policy::MergeOperation; use quickwit_indexing::TestSandbox; - use quickwit_metastore::{IndexMetadataResponseExt, ListSplitsRequestExt, SplitMetadata}; + use quickwit_metastore::{ + IndexMetadataResponseExt, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, + SplitMetadata, + }; use quickwit_proto::metastore::{DeleteQuery, IndexMetadataRequest, ListSplitsRequest}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{searcher_pool_for_test, MockSearchService}; @@ -479,7 +482,8 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .deserialize_splits_metadata() + .collect_splits_metadata() + .await .unwrap(); assert_eq!(split_metas.len(), 3); let doc_mapper = @@ -595,12 +599,13 @@ mod tests { .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits_metadata() + .await .unwrap(); - assert_eq!(all_splits[0].split_metadata.delete_opstamp, 2); - assert_eq!(all_splits[1].split_metadata.delete_opstamp, 2); + assert_eq!(all_splits[0].delete_opstamp, 2); + assert_eq!(all_splits[1].delete_opstamp, 2); // The last split has not yet its delete opstamp updated. - assert_eq!(all_splits[2].split_metadata.delete_opstamp, 0); + assert_eq!(all_splits[2].delete_opstamp, 0); test_sandbox.assert_quit().await; Ok(()) } diff --git a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs index 4ddd2c364d..4c082c36f1 100644 --- a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs +++ b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs @@ -212,6 +212,7 @@ mod tests { use quickwit_actors::Universe; use quickwit_common::shared_consts::DELETION_GRACE_PERIOD; + use quickwit_common::ServiceStream; use quickwit_metastore::{ IndexMetadata, ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitMetadata, SplitState, @@ -292,7 +293,8 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() @@ -368,7 +370,8 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() @@ -443,7 +446,8 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() @@ -618,7 +622,8 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() @@ -693,7 +698,8 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() diff --git a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs index 41ec217606..40ec1a9c1e 100644 --- a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs +++ b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs @@ -254,6 +254,7 @@ mod tests { use mockall::Sequence; use quickwit_actors::Universe; + use quickwit_common::ServiceStream; use quickwit_config::RetentionPolicy; use quickwit_metastore::{ IndexMetadata, ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitMetadata, @@ -349,7 +350,7 @@ mod tests { mock_metastore .expect_list_splits() .times(..) - .returning(|_| Ok(ListSplitsResponse::empty())); + .returning(|_| Ok(ServiceStream::empty())); mock_metastore .expect_list_indexes_metadata() .times(1) @@ -472,7 +473,8 @@ mod tests { "index-2" => Vec::new(), unknown => panic!("Unknown index: `{unknown}`."), }; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); mock_metastore diff --git a/quickwit/quickwit-janitor/src/retention_policy_execution.rs b/quickwit/quickwit-janitor/src/retention_policy_execution.rs index 6a1449c8e0..ce90cfa169 100644 --- a/quickwit/quickwit-janitor/src/retention_policy_execution.rs +++ b/quickwit/quickwit-janitor/src/retention_policy_execution.rs @@ -21,7 +21,8 @@ use quickwit_actors::ActorContext; use quickwit_common::PrettySample; use quickwit_config::RetentionPolicy; use quickwit_metastore::{ - ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitMetadata, + SplitState, }; use quickwit_proto::metastore::{ ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreService, MetastoreServiceClient, @@ -58,9 +59,9 @@ pub async fn run_execute_retention_policy( let (expired_splits, ignored_splits): (Vec, Vec) = ctx .protect_future(metastore.list_splits(list_splits_request)) .await? - .deserialize_splits()? + .collect_splits_metadata() + .await? .into_iter() - .map(|split| split.split_metadata) .partition(|split_metadata| split_metadata.time_range.is_some()); if !ignored_splits.is_empty() { diff --git a/quickwit/quickwit-metastore/Cargo.toml b/quickwit/quickwit-metastore/Cargo.toml index 96bc49a361..b0410c2ef1 100644 --- a/quickwit/quickwit-metastore/Cargo.toml +++ b/quickwit/quickwit-metastore/Cargo.toml @@ -18,6 +18,7 @@ http = { workspace = true } itertools = { workspace = true } mockall = { workspace = true, optional = true } once_cell = { workspace = true } +ouroboros = { workspace = true } rand = { workspace = true } regex = { workspace = true } sea-query = { workspace = true, optional = true } diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index 5ac68335cd..017d2c6bea 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -50,8 +50,8 @@ pub use metastore::postgresql_metastore::PostgresqlMetastore; pub use metastore::{ file_backed_metastore, AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, - ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, PublishSplitsRequestExt, - StageSplitsRequestExt, + ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, + MetastoreServiceStreamSplitsExt, PublishSplitsRequestExt, StageSplitsRequestExt, }; pub use metastore_factory::{MetastoreFactory, UnsupportedMetastore}; pub use metastore_resolver::MetastoreResolver; diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 1377886282..9cb25df928 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -30,9 +30,10 @@ use quickwit_proto::metastore::{ LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, - MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardsRequest, - OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, - ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, + OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, + StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, }; /// A [`MetastoreService`] implementation that proxies some requests to the control plane so it can @@ -144,7 +145,7 @@ impl MetastoreService for ControlPlaneMetastore { async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> MetastoreResult { + ) -> MetastoreResult> { self.metastore.list_splits(request).await } 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 068ac608c4..44d6881fb5 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -34,6 +34,7 @@ use std::time::Duration; use async_trait::async_trait; use futures::future::try_join_all; use itertools::Itertools; +use quickwit_common::ServiceStream; use quickwit_config::validate_index_id_pattern; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AcquireShardsSubrequest, AddSourceRequest, @@ -44,9 +45,9 @@ use quickwit_proto::metastore::{ ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, - MetastoreService, OpenShardsRequest, OpenShardsResponse, OpenShardsSubrequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + MetastoreService, MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, + OpenShardsSubrequest, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, + ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::IndexUid; use quickwit_storage::Storage; @@ -64,10 +65,10 @@ use self::store_operations::{ use super::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, - PublishSplitsRequestExt, StageSplitsRequestExt, + PublishSplitsRequestExt, StageSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, }; use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListSplitsQuery, MetastoreServiceExt, SplitState}; +use crate::{IndexMetadata, ListSplitsQuery, MetastoreServiceExt, Split, SplitState}; /// State of an index tracked by the metastore. pub(crate) enum IndexState { @@ -318,6 +319,30 @@ impl FileBackedMetastore { Ok(index_mutex) } + /// Returns the list of splits for the given request. + /// No error is returned if any of the requested `index_uid` does not exist. + async fn inner_list_splits(&self, request: ListSplitsRequest) -> MetastoreResult> { + let list_splits_query = request.deserialize_list_splits_query()?; + let mut all_splits = Vec::new(); + for index_uid in &list_splits_query.index_uids { + let splits = match self + .read(index_uid.clone(), |index| { + index.list_splits(&list_splits_query) + }) + .await + { + Ok(splits) => splits, + Err(MetastoreError::NotFound(_)) => { + // If the index does not exist, we just skip it. + continue; + } + Err(error) => return Err(error), + }; + all_splits.extend(splits); + } + Ok(all_splits) + } + /// Helper used for testing to obtain the data associated with the given index. #[cfg(test)] async fn get_index(&self, index_uid: IndexUid) -> MetastoreResult { @@ -610,23 +635,19 @@ impl MetastoreService for FileBackedMetastore { /// ------------------------------------------------------------------------------- /// Read-only accessors + /// Streams of splits for the given request. + /// No error is returned if any of the requested `index_uid` does not exist. async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> MetastoreResult { - let list_splits_query = request.deserialize_list_splits_query()?; - let mut all_splits = Vec::new(); - - for index_uid in &list_splits_query.index_uids { - let splits = self - .read(index_uid.clone(), |index| { - index.list_splits(&list_splits_query) - }) - .await?; - all_splits.extend(splits); - } - let response = ListSplitsResponse::try_from_splits(all_splits)?; - Ok(response) + ) -> MetastoreResult> { + let splits = self.inner_list_splits(request).await?; + let splits_responses: Vec> = splits + .chunks(STREAM_SPLITS_CHUNK_SIZE) + .map(|chunk| ListSplitsResponse::try_from_splits(chunk.to_vec())) + .collect(); + let splits_responses_stream = Box::pin(futures::stream::iter(splits_responses)); + Ok(ServiceStream::new(splits_responses_stream)) } async fn list_stale_splits( @@ -640,7 +661,8 @@ impl MetastoreService for FileBackedMetastore { .sort_by_staleness() .with_limit(request.num_splits as usize); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(list_splits_query)?; - self.list_splits(list_splits_request).await + let splits = self.inner_list_splits(list_splits_request).await?; + ListSplitsResponse::try_from_splits(splits) } async fn index_metadata( @@ -970,6 +992,7 @@ mod tests { fetch_or_init_indexes_states, meta_path, put_index_given_index_id, put_indexes_states, }; use super::*; + use crate::metastore::MetastoreServiceStreamSplitsExt; use crate::tests::DefaultForTest; use crate::{metastore_test_suite, IndexMetadata, ListSplitsQuery, SplitMetadata, SplitState}; @@ -1135,16 +1158,26 @@ mod tests { let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); - let list_splits_response = metastore.list_splits(list_splits_request).await.unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); + let splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); assert!(splits.is_empty()); let list_splits_query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(list_splits_query).unwrap(); - let list_splits_response = metastore.list_splits(list_splits_request).await.unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); + let splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); assert!(!splits.is_empty()); } @@ -1196,11 +1229,13 @@ mod tests { let create_index_response = metastore.create_index(create_index_request).await.unwrap(); let index_uid: IndexUid = create_index_response.index_uid.into(); - let list_splits_response = metastore + let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); assert!(splits.is_empty()); let split_metadata = SplitMetadata { @@ -1215,11 +1250,13 @@ mod tests { StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); metastore.stage_splits(stage_splits_request).await?; - let list_splits_response = metastore + let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); assert_eq!(splits.len(), 1); Ok(()) } @@ -1244,18 +1281,22 @@ mod tests { .unwrap(); let index_uid: IndexUid = create_index_response.index_uid.into(); - let list_splits_response = metastore_write + let splits = metastore_write .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); assert!(splits.is_empty()); - let list_splits_response = metastore_read + let splits = metastore_read .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); assert!(splits.is_empty()); let split_metadata = SplitMetadata { @@ -1270,22 +1311,25 @@ mod tests { StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); metastore_write.stage_splits(stage_splits_request).await?; - let list_splits_response = metastore_read + let splits = metastore_read .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); assert!(splits.is_empty()); for _ in 0..10 { tokio::time::sleep(polling_interval).await; - let list_splits_response = metastore_read + let splits = metastore_read .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .collect_splits() + .await .unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); - if !splits.is_empty() { return Ok(()); } @@ -1353,8 +1397,13 @@ mod tests { ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); let list_splits_request = ListSplitsRequest::try_from_list_splits_query(list_splits_query).unwrap(); - let list_splits_response = metastore.list_splits(list_splits_request).await.unwrap(); - let splits = list_splits_response.deserialize_splits().unwrap(); + let splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); // Make sure that all 20 splits are in `Published` state. assert_eq!(splits.len(), 20); diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 5b5eee3ca8..888162e0ed 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -29,6 +29,7 @@ pub mod control_plane_metastore; use std::ops::{Bound, RangeInclusive}; use async_trait::async_trait; +use futures::TryStreamExt; pub use index_metadata::IndexMetadata; use itertools::Itertools; use once_cell::sync::Lazy; @@ -39,7 +40,7 @@ use quickwit_proto::metastore::{ serde_utils, AddSourceRequest, CreateIndexRequest, DeleteTask, IndexMetadataRequest, IndexMetadataResponse, ListIndexesMetadataResponse, ListSplitsRequest, ListSplitsResponse, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, - PublishSplitsRequest, StageSplitsRequest, + MetastoreServiceStream, PublishSplitsRequest, StageSplitsRequest, }; use quickwit_proto::types::{IndexUid, SplitId}; use time::OffsetDateTime; @@ -47,6 +48,9 @@ use time::OffsetDateTime; use crate::checkpoint::IndexCheckpointDelta; use crate::{Split, SplitMetadata, SplitState}; +/// Splits batch size returned by the stream splits API +const STREAM_SPLITS_CHUNK_SIZE: usize = 1_000; + static METASTORE_METRICS_LAYER: Lazy> = Lazy::new(|| PrometheusMetricsLayer::new("metastore", ["request"])); @@ -74,6 +78,49 @@ pub trait MetastoreServiceExt: MetastoreService { impl MetastoreServiceExt for MetastoreServiceClient {} +/// Helper trait to collect splits from a [`MetastoreServiceStream`]. +#[async_trait] +pub trait MetastoreServiceStreamSplitsExt { + /// Collects all splits from a [`MetastoreServiceStream`]. + async fn collect_splits(mut self) -> MetastoreResult>; + + /// Collects all splits metadata from a [`MetastoreServiceStream`]. + async fn collect_splits_metadata(mut self) -> MetastoreResult>; + + /// Collects all splits IDs from a [`MetastoreServiceStream`]. + async fn collect_split_ids(mut self) -> MetastoreResult>; +} + +#[async_trait] +impl MetastoreServiceStreamSplitsExt for MetastoreServiceStream { + async fn collect_splits(mut self) -> MetastoreResult> { + let mut all_splits = Vec::new(); + while let Some(list_splits_response) = self.try_next().await? { + let splits = list_splits_response.deserialize_splits()?; + all_splits.extend(splits); + } + Ok(all_splits) + } + + async fn collect_splits_metadata(mut self) -> MetastoreResult> { + let mut all_splits_metadata = Vec::new(); + while let Some(list_splits_response) = self.try_next().await? { + let splits_metadata = list_splits_response.deserialize_splits_metadata()?; + all_splits_metadata.extend(splits_metadata); + } + Ok(all_splits_metadata) + } + + async fn collect_split_ids(mut self) -> MetastoreResult> { + let mut all_splits = Vec::new(); + while let Some(list_splits_response) = self.try_next().await? { + let splits = list_splits_response.deserialize_split_ids()?; + all_splits.extend(splits); + } + Ok(all_splits) + } +} + /// Helper trait to build a [`CreateIndexRequest`] and deserialize its payload. pub trait CreateIndexRequestExt { /// Creates a new [`CreateIndexRequest`] from an [`IndexConfig`]. diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index 8440a365d5..d145cb2ea0 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -17,7 +17,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::fmt::{self, Display, Write}; use std::ops::Bound; use std::str::FromStr; @@ -25,8 +25,11 @@ use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; +use futures::stream::BoxStream; +use futures::StreamExt; +use ouroboros::self_referencing; use quickwit_common::uri::Uri; -use quickwit_common::PrettySample; +use quickwit_common::{PrettySample, ServiceStream}; use quickwit_config::{ validate_index_id_pattern, MetastoreBackend, MetastoreConfig, PostgresMetastoreConfig, }; @@ -39,9 +42,10 @@ use quickwit_proto::metastore::{ LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, - MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardsRequest, - OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, - ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, + MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, + ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::IndexUid; use sea_query::{ @@ -52,9 +56,11 @@ use sqlx::migrate::Migrator; use sqlx::postgres::{PgConnectOptions, PgDatabaseError, PgPoolOptions}; use sqlx::{ConnectOptions, Pool, Postgres, Transaction}; use tokio::sync::Mutex; +use tokio_stream::Stream; use tracing::log::LevelFilter; use tracing::{debug, error, info, instrument, warn}; +use super::STREAM_SPLITS_CHUNK_SIZE; use crate::checkpoint::IndexCheckpointDelta; use crate::metastore::postgresql_model::{PgDeleteTask, PgIndex, PgSplit, Splits, ToTimestampFunc}; use crate::metastore::{instrument_metastore, FilterRange, PublishSplitsRequestExt}; @@ -802,54 +808,51 @@ impl MetastoreService for PostgresqlMetastore { async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> MetastoreResult { + ) -> MetastoreResult> { let query = request.deserialize_list_splits_query()?; let mut sql = Query::select(); sql.column(Asterisk).from(Splits::Table); append_query_filters(&mut sql, &query); let (sql, values) = sql.build_sqlx(PostgresQueryBuilder); + let split_stream = SplitStream::new( + self.connection_pool.clone(), + sql, + |connection_pool: &Pool, sql: &String| { + sqlx::query_as_with::<_, PgSplit, _>(sql, values).fetch(connection_pool) + }, + ); - let pg_splits = sqlx::query_as_with::<_, PgSplit, _>(&sql, values) - .fetch_all(&self.connection_pool) - .await?; - - // If no splits were returned, maybe some indexes do not exist in the first place? - // TODO: the file-backed metastore is more accurate as it checks for index existence before - // returning splits. We could do the same here or remove index existence check `list_splits` - // for all metastore implementations. - if pg_splits.is_empty() { - let index_ids_str: Vec = query - .index_uids - .iter() - .map(|index_uid| index_uid.index_id().to_string()) - .collect(); - let list_indexes_metadata_request = ListIndexesMetadataRequest { - index_id_patterns: index_ids_str.clone(), - }; - let found_index_ids: HashSet = self - .list_indexes_metadata(list_indexes_metadata_request) - .await? - .deserialize_indexes_metadata()? - .into_iter() - .map(|index_metadata| index_metadata.index_id().to_string()) - .collect(); - let not_found_index_ids: Vec = index_ids_str - .into_iter() - .filter(|index_id| !found_index_ids.contains(index_id)) - .collect(); - if !not_found_index_ids.is_empty() { - return Err(MetastoreError::NotFound(EntityKind::Indexes { - index_ids: not_found_index_ids, - })); - } - } - let splits = pg_splits - .into_iter() - .map(|pg_split| pg_split.try_into()) - .collect::>>()?; - let response = ListSplitsResponse::try_from_splits(splits)?; - Ok(response) + let mapped_split_stream = + split_stream + .chunks(STREAM_SPLITS_CHUNK_SIZE) + .map(|pg_splits_res| { + let mut splits = Vec::with_capacity(pg_splits_res.len()); + for pg_split_res in pg_splits_res { + let pg_split = match pg_split_res { + Ok(pg_split) => pg_split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to fetch splits".to_string(), + cause: error.to_string(), + }) + } + }; + let split: Split = match pg_split.try_into() { + Ok(split) => split, + Err(error) => { + return Err(MetastoreError::Internal { + message: "failed to convert `PgSplit` into `Split`".to_string(), + cause: error.to_string(), + }) + } + }; + splits.push(split); + } + ListSplitsResponse::try_from_splits(splits) + }); + let service_stream = ServiceStream::new(Box::pin(mapped_split_stream)); + Ok(service_stream) } #[instrument(skip(self))] @@ -1271,16 +1274,6 @@ impl MetastoreService for PostgresqlMetastore { .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::NotFound(EntityKind::Index { - index_id: index_uid.index_id().to_string(), - })); - } let splits = pg_stale_splits .into_iter() .map(|pg_split| pg_split.try_into()) @@ -1320,6 +1313,28 @@ impl MetastoreService for PostgresqlMetastore { impl MetastoreServiceExt for PostgresqlMetastore {} +#[self_referencing] +struct SplitStream { + connection_pool: Pool, + sql: String, + #[borrows(connection_pool, sql)] + #[covariant] + inner: BoxStream<'this, Result>, +} + +impl Stream for SplitStream { + type Item = Result; + + fn poll_next( + mut self: std::pin::Pin<&mut Self>, + cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + SplitStream::with_inner_mut(&mut self, |this| { + std::pin::Pin::new(&mut this.as_mut()).poll_next(cx) + }) + } +} + // We use dollar-quoted strings in Postgresql. // // In order to ensure that we do not risk SQL injection, diff --git a/quickwit/quickwit-metastore/src/tests/delete_task.rs b/quickwit/quickwit-metastore/src/tests/delete_task.rs new file mode 100644 index 0000000000..82d253d60b --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/delete_task.rs @@ -0,0 +1,288 @@ +// 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 quickwit_common::rand::append_random_suffix; +use quickwit_config::IndexConfig; +use quickwit_proto::metastore::{ + CreateIndexRequest, DeleteIndexRequest, DeleteQuery, EntityKind, LastDeleteOpstampRequest, + ListDeleteTasksRequest, MetastoreError, +}; +use quickwit_proto::types::IndexUid; +use quickwit_query::query_ast::qast_json_helper; + +use super::DefaultForTest; +use crate::tests::cleanup_index; +use crate::{CreateIndexRequestExt, MetastoreServiceExt}; + +pub async fn test_metastore_create_delete_task< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let index_id = append_random_suffix("add-delete-task"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + let delete_query = DeleteQuery { + index_uid: index_uid.clone().into(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + + // Create a delete task on non-existing index. + let error = metastore + .create_delete_task(DeleteQuery { + index_uid: IndexUid::new_with_random_ulid("does-not-exist").to_string(), + ..delete_query.clone() + }) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + // Create a delete task on an index with wrong incarnation_id + let error = metastore + .create_delete_task(DeleteQuery { + index_uid: IndexUid::from_parts(&index_id, "12345").to_string(), + ..delete_query.clone() + }) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + // Create a delete task. + let delete_task_1 = metastore + .create_delete_task(delete_query.clone()) + .await + .unwrap(); + assert!(delete_task_1.opstamp > 0); + let delete_query_1 = delete_task_1.delete_query.unwrap(); + assert_eq!(delete_query_1.index_uid, delete_query.index_uid); + assert_eq!(delete_query_1.start_timestamp, delete_query.start_timestamp); + assert_eq!(delete_query_1.end_timestamp, delete_query.end_timestamp); + let delete_task_2 = metastore + .create_delete_task(delete_query.clone()) + .await + .unwrap(); + assert!(delete_task_2.opstamp > delete_task_1.opstamp); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_last_delete_opstamp< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let index_id_1 = append_random_suffix("test-last-delete-opstamp-1"); + let index_uri_1 = format!("ram:///indexes/{index_id_1}"); + let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); + let index_id_2 = append_random_suffix("test-last-delete-opstamp-2"); + let index_uri_2 = format!("ram:///indexes/{index_id_2}"); + let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); + let index_uid_1: IndexUid = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_2: IndexUid = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap()) + .await + .unwrap() + .index_uid + .into(); + + let delete_query_index_1 = DeleteQuery { + index_uid: index_uid_1.to_string(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + let delete_query_index_2 = DeleteQuery { + index_uid: index_uid_2.to_string(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + + let last_opstamp_index_1_with_no_task = metastore + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_1.to_string(), + }) + .await + .unwrap() + .last_delete_opstamp; + assert_eq!(last_opstamp_index_1_with_no_task, 0); + + // Create a delete task. + metastore + .create_delete_task(delete_query_index_1.clone()) + .await + .unwrap(); + let delete_task_2 = metastore + .create_delete_task(delete_query_index_1.clone()) + .await + .unwrap(); + let delete_task_3 = metastore + .create_delete_task(delete_query_index_2.clone()) + .await + .unwrap(); + + let last_opstamp_index_1 = metastore + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_1.to_string(), + }) + .await + .unwrap() + .last_delete_opstamp; + let last_opstamp_index_2 = metastore + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_2.to_string(), + }) + .await + .unwrap() + .last_delete_opstamp; + assert_eq!(last_opstamp_index_1, delete_task_2.opstamp); + assert_eq!(last_opstamp_index_2, delete_task_3.opstamp); + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; +} + +pub async fn test_metastore_delete_index_with_tasks< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let index_id = append_random_suffix("delete-delete-tasks"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + let delete_query = DeleteQuery { + index_uid: index_uid.clone().into(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + let _ = metastore + .create_delete_task(delete_query.clone()) + .await + .unwrap(); + let _ = metastore + .create_delete_task(delete_query.clone()) + .await + .unwrap(); + + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.clone().into(), + }) + .await + .unwrap(); +} + +pub async fn test_metastore_list_delete_tasks< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let index_id_1 = append_random_suffix("test-list-delete-tasks-1"); + let index_uri_1 = format!("ram:///indexes/{index_id_1}"); + let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); + let index_id_2 = append_random_suffix("test-list-delete-tasks-2"); + let index_uri_2 = format!("ram:///indexes/{index_id_2}"); + let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); + let index_uid_1: IndexUid = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_2: IndexUid = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let delete_query_index_1 = DeleteQuery { + index_uid: index_uid_1.to_string(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + let delete_query_index_2 = DeleteQuery { + index_uid: index_uid_2.to_string(), + query_ast: qast_json_helper("my_field:my_value", &[]), + start_timestamp: Some(1), + end_timestamp: Some(2), + }; + + // Create a delete task. + let delete_task_1 = metastore + .create_delete_task(delete_query_index_1.clone()) + .await + .unwrap(); + let delete_task_2 = metastore + .create_delete_task(delete_query_index_1.clone()) + .await + .unwrap(); + let _ = metastore + .create_delete_task(delete_query_index_2.clone()) + .await + .unwrap(); + + let all_index_id_1_delete_tasks = metastore + .list_delete_tasks(ListDeleteTasksRequest::new(index_uid_1.clone(), 0)) + .await + .unwrap() + .delete_tasks; + assert_eq!(all_index_id_1_delete_tasks.len(), 2); + + let recent_index_id_1_delete_tasks = metastore + .list_delete_tasks(ListDeleteTasksRequest::new( + index_uid_1.clone(), + delete_task_1.opstamp, + )) + .await + .unwrap() + .delete_tasks; + assert_eq!(recent_index_id_1_delete_tasks.len(), 1); + assert_eq!( + recent_index_id_1_delete_tasks[0].opstamp, + delete_task_2.opstamp + ); + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; +} diff --git a/quickwit/quickwit-metastore/src/tests/index.rs b/quickwit/quickwit-metastore/src/tests/index.rs new file mode 100644 index 0000000000..4b868ee8bf --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/index.rs @@ -0,0 +1,375 @@ +// 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 . + +// Index API tests +// +// - create_index +// - index_exists +// - index_metadata +// - list_indexes +// - delete_index + +use quickwit_common::rand::append_random_suffix; +use quickwit_config::IndexConfig; +use quickwit_proto::metastore::{ + CreateIndexRequest, DeleteIndexRequest, EntityKind, IndexMetadataRequest, + ListIndexesMetadataRequest, MetastoreError, MetastoreService, StageSplitsRequest, +}; +use quickwit_proto::types::IndexUid; + +use super::DefaultForTest; +use crate::tests::cleanup_index; +use crate::{ + CreateIndexRequestExt, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, + MetastoreServiceExt, SplitMetadata, StageSplitsRequestExt, +}; + +pub async fn test_metastore_create_index< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-create-index"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request.clone()) + .await + .unwrap() + .index_uid + .into(); + + assert!(metastore.index_exists(&index_id).await.unwrap()); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + + assert_eq!(index_metadata.index_id(), index_id); + assert_eq!(index_metadata.index_uri(), &index_uri); + + let error = metastore + .create_index(create_index_request) + .await + .unwrap_err(); + assert!(matches!(error, MetastoreError::AlreadyExists { .. })); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_create_index_with_maximum_length< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix(format!("very-long-index-{}", "a".repeat(233)).as_str()); + assert_eq!(index_id.len(), 255); + let index_uri = format!("ram:///indexes/{index_id}"); + + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + assert!(metastore.index_exists(&index_id).await.unwrap()); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_index_exists< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-index-exists"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + + assert!(!metastore.index_exists(&index_id).await.unwrap()); + + let index_uid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + assert!(metastore.index_exists(&index_id).await.unwrap()); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_index_metadata< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-index-metadata"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let error = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + + assert_eq!(index_metadata.index_id(), index_id); + assert_eq!(index_metadata.index_uri(), &index_uri); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_list_all_indexes< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id_prefix = append_random_suffix("test-list-all-indexes"); + let index_id_1 = format!("{index_id_prefix}-1"); + let index_uri_1 = format!("ram:///indexes/{index_id_1}"); + let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); + + let index_id_2 = format!("{index_id_prefix}-2"); + let index_uri_2 = format!("ram:///indexes/{index_id_2}"); + let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); + let indexes_count = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) + .await + .unwrap() + .deserialize_indexes_metadata() + .unwrap() + .into_iter() + .filter(|index| index.index_id().starts_with(&index_id_prefix)) + .count(); + assert_eq!(indexes_count, 0); + + let index_uid_1 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_1).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_2 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_2).unwrap()) + .await + .unwrap() + .index_uid + .into(); + + let indexes_count = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) + .await + .unwrap() + .deserialize_indexes_metadata() + .unwrap() + .into_iter() + .filter(|index| index.index_id().starts_with(&index_id_prefix)) + .count(); + assert_eq!(indexes_count, 2); + + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; +} + +pub async fn test_metastore_list_indexes() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id_fragment = append_random_suffix("test-list-indexes"); + let index_id_1 = format!("prefix-1-{index_id_fragment}-suffix-1"); + let index_uri_1 = format!("ram:///indexes/{index_id_1}"); + let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); + + let index_id_2 = format!("prefix-2-{index_id_fragment}-suffix-2"); + let index_uri_2 = format!("ram:///indexes/{index_id_2}"); + let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); + + let index_id_3 = format!("prefix.3.{index_id_fragment}.3"); + let index_uri_3 = format!("ram:///indexes/{index_id_3}"); + let index_config_3 = IndexConfig::for_test(&index_id_3, &index_uri_3); + + let index_id_4 = format!("p-4-{index_id_fragment}-suffix-4"); + let index_uri_4 = format!("ram:///indexes/{index_id_4}"); + let index_config_4 = IndexConfig::for_test(&index_id_4, &index_uri_4); + + let index_id_patterns = vec![ + format!("prefix-*-{index_id_fragment}-suffix-*"), + format!("prefix*{index_id_fragment}*suffix-*"), + ]; + let indexes_count = metastore + .list_indexes_metadata(ListIndexesMetadataRequest { index_id_patterns }) + .await + .unwrap() + .deserialize_indexes_metadata() + .unwrap() + .len(); + assert_eq!(indexes_count, 0); + + let index_uid_1 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_1).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_2 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_2).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_3 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_3).unwrap()) + .await + .unwrap() + .index_uid + .into(); + let index_uid_4 = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config_4).unwrap()) + .await + .unwrap() + .index_uid + .into(); + + let index_id_patterns = vec![format!("prefix-*-{index_id_fragment}-suffix-*")]; + let indexes_count = metastore + .list_indexes_metadata(ListIndexesMetadataRequest { index_id_patterns }) + .await + .unwrap() + .deserialize_indexes_metadata() + .unwrap() + .len(); + assert_eq!(indexes_count, 2); + + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; + cleanup_index(&mut metastore, index_uid_3).await; + cleanup_index(&mut metastore, index_uid_4).await; +} + +pub async fn test_metastore_delete_index< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-delete-index"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let index_uid_not_existing = IndexUid::new_with_random_ulid("index-not-found"); + let error = metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid_not_existing.to_string(), + }) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let error = metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid_not_existing.to_string(), + }) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.clone().into(), + }) + .await + .unwrap(); + + assert!(!metastore.index_exists(&index_id).await.unwrap()); + + let split_id = format!("{index_id}--split"); + let split_metadata = SplitMetadata { + split_id: split_id.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + // TODO: We should not be able to delete an index that has remaining splits, at least not as + // 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(); + // assert_eq!(splits.len(), 1) + + cleanup_index(&mut metastore, index_uid).await; +} diff --git a/quickwit/quickwit-metastore/src/tests/list_splits.rs b/quickwit/quickwit-metastore/src/tests/list_splits.rs new file mode 100644 index 0000000000..9d4ac809be --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/list_splits.rs @@ -0,0 +1,1089 @@ +// 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::time::Duration; + +use futures::TryStreamExt; +use itertools::Itertools; +use quickwit_common::rand::append_random_suffix; +use quickwit_config::IndexConfig; +use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; +use quickwit_proto::metastore::{ + CreateIndexRequest, ListSplitsRequest, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, + PublishSplitsRequest, StageSplitsRequest, +}; +use quickwit_proto::types::{IndexUid, SplitId}; +use time::OffsetDateTime; +use tokio::time::sleep; +use tracing::info; + +use super::{to_btree_set, DefaultForTest}; +use crate::metastore::MetastoreServiceStreamSplitsExt; +use crate::tests::{cleanup_index, collect_split_ids}; +use crate::{ + CreateIndexRequestExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, + MetastoreServiceExt, SplitMaturity, SplitMetadata, SplitState, StageSplitsRequestExt, +}; + +pub async fn test_metastore_list_all_splits< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-list-all-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let split_id_4 = format!("{index_id}--split-4"); + let split_metadata_4 = SplitMetadata { + split_id: split_id_4.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let split_id_5 = format!("{index_id}--split-5"); + let split_metadata_5 = SplitMetadata { + split_id: split_id_5.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let split_id_6 = format!("{index_id}--split-6"); + let split_metadata_6 = SplitMetadata { + split_id: split_id_6.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + + let no_splits = metastore + .list_splits( + ListSplitsRequest::try_from_index_uid(IndexUid::new_with_random_ulid( + "index-not-found", + )) + .unwrap(), + ) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + assert!(no_splits.is_empty()); + + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![ + split_metadata_1, + split_metadata_2, + split_metadata_3, + split_metadata_4, + split_metadata_5, + split_metadata_6, + ], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let mark_splits_for_deletion = MarkSplitsForDeletionRequest::new( + index_uid.clone(), + vec![split_id_3.clone(), split_id_4.clone()], + ); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion) + .await + .unwrap(); + + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5, + &split_id_6 + ] + ); + + cleanup_index(&mut metastore, index_uid.clone()).await; +} + +pub async fn test_metastore_stream_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-stream-splits"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let mut split_metadatas_to_create = Vec::new(); + for idx in 1..10001 { + let split_id = format!("{index_id}--split-{idx:0>5}"); + let split_metadata = SplitMetadata { + split_id: split_id.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + split_metadatas_to_create.push(split_metadata); + + if idx > 0 && idx % 1000 == 0 { + let staged_split_ids: Vec = split_metadatas_to_create + .iter() + .map(|split_metadata| split_metadata.split_id.clone()) + .collect(); + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + split_metadatas_to_create.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids, + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + split_metadatas_to_create.clear(); + } + } + + let stream_splits_request = ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap(); + let mut stream_response = metastore.list_splits(stream_splits_request).await.unwrap(); + let mut all_splits = Vec::new(); + for _ in 0..10 { + let mut splits = stream_response + .try_next() + .await + .unwrap() + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 1000); + all_splits.append(&mut splits); + } + all_splits.sort_by_key(|split| split.split_id().to_string()); + assert_eq!(all_splits[0].split_id(), format!("{index_id}--split-00001")); + assert_eq!( + all_splits[all_splits.len() - 1].split_id(), + format!("{index_id}--split-10000") + ); +} + +pub async fn test_metastore_list_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + + let index_id = append_random_suffix("test-list-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + time_range: Some(0..=99), + create_timestamp: current_timestamp, + maturity: SplitMaturity::Immature { + maturation_period: Duration::from_secs(0), + }, + tags: to_btree_set(&["tag!", "tag:foo", "$tag!", "$tag:bar"]), + delete_opstamp: 3, + ..Default::default() + }; + + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + time_range: Some(100..=199), + create_timestamp: current_timestamp, + maturity: SplitMaturity::Immature { + maturation_period: Duration::from_secs(10), + }, + tags: to_btree_set(&["tag!", "$tag!", "$tag:bar"]), + delete_opstamp: 1, + ..Default::default() + }; + + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + time_range: Some(200..=299), + create_timestamp: current_timestamp, + maturity: SplitMaturity::Immature { + maturation_period: Duration::from_secs(20), + }, + tags: to_btree_set(&["tag!", "tag:foo", "tag:baz", "$tag!"]), + delete_opstamp: 5, + ..Default::default() + }; + + let split_id_4 = format!("{index_id}--split-4"); + let split_metadata_4 = SplitMetadata { + split_id: split_id_4.clone(), + index_uid: index_uid.clone(), + time_range: Some(300..=399), + tags: to_btree_set(&["tag!", "tag:foo", "$tag!"]), + delete_opstamp: 7, + ..Default::default() + }; + + let split_id_5 = format!("{index_id}--split-5"); + let split_metadata_5 = SplitMetadata { + split_id: split_id_5.clone(), + index_uid: index_uid.clone(), + time_range: None, + create_timestamp: current_timestamp, + tags: to_btree_set(&["tag!", "tag:baz", "tag:biz", "$tag!"]), + delete_opstamp: 9, + ..Default::default() + }; + + { + let query = + ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + assert!(splits.is_empty()); + } + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![ + split_metadata_1.clone(), + split_metadata_2.clone(), + split_metadata_3.clone(), + split_metadata_4.clone(), + split_metadata_5.clone(), + ], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let query = ListSplitsQuery::for_index(index_uid.clone()).with_limit(3); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + assert_eq!( + splits.len(), + 3, + "Expected number of splits returned to match limit.", + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()).with_offset(3); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + assert_eq!( + splits.len(), + 2, + "Expected 3 splits to be skipped out of the 5 provided splits.", + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(99); + + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids: Vec<&str> = splits + .iter() + .map(|split| split.split_id()) + .sorted() + .collect(); + assert_eq!(split_ids, &[&split_id_1, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(200); + + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_end_lt(200); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(100); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(101); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(199); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(200); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(201); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(299); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(300); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(0) + .with_time_range_end_lt(301); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5 + ] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(301) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(300) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(299) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(201) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(200) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(199) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(101) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(101) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(100) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + + assert_eq!( + split_ids, + &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(99) + .with_time_range_end_lt(400); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5 + ] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_time_range_start_gte(1000) + .with_time_range_end_lt(1100); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_5]); + + // Artificially increase the create_timestamp + sleep(Duration::from_secs(1)).await; + // add a split without tag + let split_id_6 = format!("{index_id}--split-6"); + let split_metadata_6 = SplitMetadata { + split_id: split_id_6.clone(), + index_uid: index_uid.clone(), + time_range: None, + create_timestamp: OffsetDateTime::now_utc().unix_timestamp(), + ..Default::default() + }; + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_6.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let query = + ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5, + &split_id_6, + ] + ); + + let tag_filter_ast = TagFilterAst::Or(vec![ + TagFilterAst::Or(vec![no_tag("$tag!"), tag("$tag:bar")]), + TagFilterAst::Or(vec![no_tag("tag!"), tag("tag:baz")]), + ]); + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::Staged) + .with_tags_filter(tag_filter_ast); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_5, + &split_id_6, + ] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_update_timestamp_gte(current_timestamp); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5, + &split_id_6, + ] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_update_timestamp_gte(split_metadata_6.create_timestamp); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids: Vec<&String> = splits + .iter() + .map(|split| &split.split_metadata.split_id) + .sorted() + .collect(); + assert_eq!(split_ids, vec![&split_id_6]); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .with_create_timestamp_lt(split_metadata_6.create_timestamp); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[ + &split_id_1, + &split_id_2, + &split_id_3, + &split_id_4, + &split_id_5, + ] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()).with_delete_opstamp_lt(6); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_1, &split_id_2, &split_id_3, &split_id_6,] + ); + + // Test maturity filter + let maturity_evaluation_timestamp = + OffsetDateTime::from_unix_timestamp(current_timestamp).unwrap(); + let query = ListSplitsQuery::for_index(index_uid.clone()) + .retain_mature(maturity_evaluation_timestamp); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!( + split_ids, + &[&split_id_1, &split_id_4, &split_id_5, &split_id_6,] + ); + + let query = ListSplitsQuery::for_index(index_uid.clone()) + .retain_immature(maturity_evaluation_timestamp); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_2, &split_id_3]); + + cleanup_index(&mut metastore, index_uid).await; + } +} + +pub async fn test_metastore_list_stale_splits< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + let index_id = append_random_suffix("test-list-stale-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 20, + ..Default::default() + }; + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 10, + ..Default::default() + }; + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 0, + ..Default::default() + }; + let split_id_4 = format!("{index_id}--split-4"); + let split_metadata_4 = SplitMetadata { + split_id: split_id_4.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 20, + ..Default::default() + }; + // immature split + let split_id_5 = format!("{index_id}--split-5"); + let split_metadata_5 = SplitMetadata { + split_id: split_id_5.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + maturity: SplitMaturity::Immature { + maturation_period: Duration::from_secs(100), + }, + delete_opstamp: 0, + ..Default::default() + }; + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + delete_opstamp: 0, + num_splits: 100, + }; + let no_splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert!(no_splits.is_empty()); + + { + info!("list stale splits on an index"); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![ + split_metadata_1.clone(), + split_metadata_2.clone(), + split_metadata_3.clone(), + split_metadata_5.clone(), + ], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + // Sleep for 1 second to have different publish timestamps. + sleep(Duration::from_secs(1)).await; + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_4.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id_4.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + // Sleep for 1 second to have different publish timestamps. + tokio::time::sleep(Duration::from_secs(1)).await; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone(), split_id_5.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 100, + num_splits: 1, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 1); + assert_eq!( + splits[0].split_metadata.delete_opstamp, + split_metadata_2.delete_opstamp + ); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 100, + num_splits: 4, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 3); + assert_eq!(splits[0].split_id(), split_metadata_2.split_id()); + assert_eq!(splits[1].split_id(), split_metadata_4.split_id()); + assert_eq!(splits[2].split_id(), split_metadata_1.split_id()); + assert_eq!( + splits[2].split_metadata.delete_opstamp, + split_metadata_1.delete_opstamp + ); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 20, + num_splits: 2, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 1); + assert_eq!( + splits[0].split_metadata.delete_opstamp, + split_metadata_2.delete_opstamp + ); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 10, + num_splits: 2, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert!(splits.is_empty()); + cleanup_index(&mut metastore, index_uid).await; + } +} diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index 64033d632c..4cc6caceb2 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -18,3834 +18,149 @@ // along with this program. If not, see . use std::collections::BTreeSet; -use std::num::NonZeroUsize; -use std::time::Duration; use async_trait::async_trait; -use futures::future::try_join_all; use itertools::Itertools; -use quickwit_common::rand::append_random_suffix; -use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; -use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; +use quickwit_proto::metastore::metastore_service_grpc_client::MetastoreServiceGrpcClient; use quickwit_proto::metastore::{ - AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, DeleteQuery, DeleteSourceRequest, - DeleteSplitsRequest, EntityKind, IndexMetadataRequest, LastDeleteOpstampRequest, - ListDeleteTasksRequest, ListIndexesMetadataRequest, ListSplitsRequest, ListStaleSplitsRequest, - MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, PublishSplitsRequest, - ResetSourceCheckpointRequest, SourceType, StageSplitsRequest, ToggleSourceRequest, - UpdateSplitsDeleteOpstampRequest, + DeleteIndexRequest, DeleteSplitsRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, + MetastoreServiceClient, MetastoreServiceGrpcClientAdapter, }; -use quickwit_proto::types::{IndexUid, Position}; -use quickwit_query::query_ast::qast_json_helper; -use time::OffsetDateTime; -use tokio::time::sleep; -use tracing::{error, info}; +use quickwit_proto::tonic::transport::Channel; +use quickwit_proto::types::IndexUid; +pub(crate) mod delete_task; +pub(crate) mod index; +pub(crate) mod list_splits; pub(crate) mod shard; +pub(crate) mod source; +pub(crate) mod split; -use crate::checkpoint::{ - IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, -}; -use crate::{ - AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, - ListIndexesMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, - MetastoreServiceExt, Split, SplitMaturity, SplitMetadata, SplitState, StageSplitsRequestExt, -}; - -#[async_trait] -pub trait DefaultForTest { - async fn default_for_test() -> Self; -} - -fn collect_split_ids(splits: &[Split]) -> Vec<&str> { - splits - .iter() - .map(|split| split.split_id()) - .sorted() - .collect() -} - -fn to_btree_set(tags: &[&str]) -> BTreeSet { - tags.iter().map(|tag| tag.to_string()).collect() -} - -async fn cleanup_index(metastore: &mut dyn MetastoreService, index_uid: IndexUid) { - // List all splits. - let all_splits = metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - - if !all_splits.is_empty() { - let all_split_ids: Vec = all_splits - .iter() - .map(|split| split.split_id().to_string()) - .collect(); - - // Mark splits for deletion. - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), all_split_ids.clone()); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - // Delete splits. - let delete_splits_request = DeleteSplitsRequest { - index_uid: index_uid.clone().into(), - split_ids: all_split_ids, - }; - metastore - .delete_splits(delete_splits_request) - .await - .unwrap(); - } - // Delete index. - metastore - .delete_index(DeleteIndexRequest { - index_uid: index_uid.clone().into(), - }) - .await - .unwrap(); -} - -// Index API tests -// -// - create_index -// - index_exists -// - index_metadata -// - list_indexes -// - delete_index - -pub async fn test_metastore_create_index< - MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-create-index"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request.clone()) - .await - .unwrap() - .index_uid - .into(); - - assert!(metastore.index_exists(&index_id).await.unwrap()); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - - assert_eq!(index_metadata.index_id(), index_id); - assert_eq!(index_metadata.index_uri(), &index_uri); - - let error = metastore - .create_index(create_index_request) - .await - .unwrap_err(); - assert!(matches!(error, MetastoreError::AlreadyExists { .. })); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_create_index_with_maximum_length< - MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix(format!("very-long-index-{}", "a".repeat(233)).as_str()); - assert_eq!(index_id.len(), 255); - let index_uri = format!("ram:///indexes/{index_id}"); - - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - assert!(metastore.index_exists(&index_id).await.unwrap()); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_index_exists< - MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-index-exists"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - - assert!(!metastore.index_exists(&index_id).await.unwrap()); - - let index_uid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - assert!(metastore.index_exists(&index_id).await.unwrap()); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_index_metadata() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-index-metadata"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let error = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - - assert_eq!(index_metadata.index_id(), index_id); - assert_eq!(index_metadata.index_uri(), &index_uri); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_list_all_indexes() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id_prefix = append_random_suffix("test-list-all-indexes"); - let index_id_1 = format!("{index_id_prefix}-1"); - let index_uri_1 = format!("ram:///indexes/{index_id_1}"); - let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); - - let index_id_2 = format!("{index_id_prefix}-2"); - let index_uri_2 = format!("ram:///indexes/{index_id_2}"); - let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); - let indexes_count = metastore - .list_indexes_metadata(ListIndexesMetadataRequest::all()) - .await - .unwrap() - .deserialize_indexes_metadata() - .unwrap() - .into_iter() - .filter(|index| index.index_id().starts_with(&index_id_prefix)) - .count(); - assert_eq!(indexes_count, 0); - - let index_uid_1 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_1).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_2 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_2).unwrap()) - .await - .unwrap() - .index_uid - .into(); - - let indexes_count = metastore - .list_indexes_metadata(ListIndexesMetadataRequest::all()) - .await - .unwrap() - .deserialize_indexes_metadata() - .unwrap() - .into_iter() - .filter(|index| index.index_id().starts_with(&index_id_prefix)) - .count(); - assert_eq!(indexes_count, 2); - - cleanup_index(&mut metastore, index_uid_1).await; - cleanup_index(&mut metastore, index_uid_2).await; -} - -pub async fn test_metastore_list_indexes() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id_fragment = append_random_suffix("test-list-indexes"); - let index_id_1 = format!("prefix-1-{index_id_fragment}-suffix-1"); - let index_uri_1 = format!("ram:///indexes/{index_id_1}"); - let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); - - let index_id_2 = format!("prefix-2-{index_id_fragment}-suffix-2"); - let index_uri_2 = format!("ram:///indexes/{index_id_2}"); - let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); - - let index_id_3 = format!("prefix.3.{index_id_fragment}.3"); - let index_uri_3 = format!("ram:///indexes/{index_id_3}"); - let index_config_3 = IndexConfig::for_test(&index_id_3, &index_uri_3); - - let index_id_4 = format!("p-4-{index_id_fragment}-suffix-4"); - let index_uri_4 = format!("ram:///indexes/{index_id_4}"); - let index_config_4 = IndexConfig::for_test(&index_id_4, &index_uri_4); - - let index_id_patterns = vec![ - format!("prefix-*-{index_id_fragment}-suffix-*"), - format!("prefix*{index_id_fragment}*suffix-*"), - ]; - let indexes_count = metastore - .list_indexes_metadata(ListIndexesMetadataRequest { index_id_patterns }) - .await - .unwrap() - .deserialize_indexes_metadata() - .unwrap() - .len(); - assert_eq!(indexes_count, 0); - - let index_uid_1 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_1).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_2 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_2).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_3 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_3).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_4 = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_4).unwrap()) - .await - .unwrap() - .index_uid - .into(); - - let index_id_patterns = vec![format!("prefix-*-{index_id_fragment}-suffix-*")]; - let indexes_count = metastore - .list_indexes_metadata(ListIndexesMetadataRequest { index_id_patterns }) - .await - .unwrap() - .deserialize_indexes_metadata() - .unwrap() - .len(); - assert_eq!(indexes_count, 2); - - cleanup_index(&mut metastore, index_uid_1).await; - cleanup_index(&mut metastore, index_uid_2).await; - cleanup_index(&mut metastore, index_uid_3).await; - cleanup_index(&mut metastore, index_uid_4).await; -} - -pub async fn test_metastore_delete_index< - MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-delete-index"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let index_uid_not_existing = IndexUid::new_with_random_ulid("index-not-found"); - let error = metastore - .delete_index(DeleteIndexRequest { - index_uid: index_uid_not_existing.to_string(), - }) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let error = metastore - .delete_index(DeleteIndexRequest { - index_uid: index_uid_not_existing.to_string(), - }) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - metastore - .delete_index(DeleteIndexRequest { - index_uid: index_uid.clone().into(), - }) - .await - .unwrap(); - - assert!(!metastore.index_exists(&index_id).await.unwrap()); - - let split_id = format!("{index_id}--split"); - let split_metadata = SplitMetadata { - split_id: split_id.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - - let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - // TODO: We should not be able to delete an index that has remaining splits, at least not as - // 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(); - // assert_eq!(splits.len(), 1) - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_add_source() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-add-source"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let source_id = format!("{index_id}--source"); - - let source = SourceConfig { - source_id: source_id.to_string(), - max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), - desired_num_pipelines: NonZeroUsize::new(1).unwrap(), - enabled: true, - source_params: SourceParams::void(), - transform_config: None, - input_format: SourceInputFormat::Json, - }; - - assert_eq!( - metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap() - .checkpoint - .source_checkpoint(&source_id), - None - ); - - let add_source_request = - AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(); - metastore.add_source(add_source_request).await.unwrap(); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - - let sources = &index_metadata.sources; - assert_eq!(sources.len(), 1); - assert!(sources.contains_key(&source_id)); - assert_eq!(sources.get(&source_id).unwrap().source_id, source_id); - assert_eq!( - sources.get(&source_id).unwrap().source_type(), - SourceType::Void - ); - assert_eq!( - index_metadata.checkpoint.source_checkpoint(&source_id), - Some(&SourceCheckpoint::default()) - ); - - assert!(matches!( - metastore - .add_source( - AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) - .unwrap() - ) - .await - .unwrap_err(), - MetastoreError::AlreadyExists(EntityKind::Source { .. }) - )); - assert!(matches!( - metastore - .add_source( - AddSourceRequest::try_from_source_config( - IndexUid::new_with_random_ulid("index-not-found"), - source.clone() - ) - .unwrap() - ) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - assert!(matches!( - metastore - .add_source( - AddSourceRequest::try_from_source_config( - IndexUid::new_with_random_ulid(&index_id), - source - ) - .unwrap() - ) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_toggle_source() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-toggle-source"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let source_id = format!("{index_id}--source"); - let source = SourceConfig { - source_id: source_id.to_string(), - max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), - desired_num_pipelines: NonZeroUsize::new(1).unwrap(), - enabled: true, - source_params: SourceParams::void(), - transform_config: None, - input_format: SourceInputFormat::Json, - }; - let add_source_request = - AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(); - metastore.add_source(add_source_request).await.unwrap(); - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let source = index_metadata.sources.get(&source_id).unwrap(); - assert_eq!(source.enabled, true); - - // Disable source. - metastore - .toggle_source(ToggleSourceRequest { - index_uid: index_uid.clone().into(), - source_id: source.source_id.clone(), - enable: false, - }) - .await - .unwrap(); - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let source = index_metadata.sources.get(&source_id).unwrap(); - assert_eq!(source.enabled, false); - - // Enable source. - metastore - .toggle_source(ToggleSourceRequest { - index_uid: index_uid.clone().into(), - source_id: source.source_id.clone(), - enable: true, - }) - .await - .unwrap(); - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let source = index_metadata.sources.get(&source_id).unwrap(); - assert_eq!(source.enabled, true); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_delete_source() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-delete-source"); - let index_uri = format!("ram:///indexes/{index_id}"); - let source_id = format!("{index_id}--source"); - - let source = SourceConfig { - source_id: source_id.to_string(), - max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), - desired_num_pipelines: NonZeroUsize::new(1).unwrap(), - enabled: true, - source_params: SourceParams::void(), - transform_config: None, - input_format: SourceInputFormat::Json, - }; - - let index_config = IndexConfig::for_test(&index_id, index_uri.as_str()); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - assert!(matches!( - metastore - .add_source( - AddSourceRequest::try_from_source_config( - IndexUid::new_with_random_ulid("index-not-found"), - source.clone() - ) - .unwrap() - ) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - assert!(matches!( - metastore - .add_source( - AddSourceRequest::try_from_source_config( - IndexUid::new_with_random_ulid(&index_id), - source.clone() - ) - .unwrap() - ) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - metastore - .add_source( - AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(), - ) - .await - .unwrap(); - metastore - .delete_source(DeleteSourceRequest { - index_uid: index_uid.clone().into(), - source_id: source_id.clone(), - }) - .await - .unwrap(); - - let sources = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap() - .sources; - assert!(sources.is_empty()); - - assert!(matches!( - metastore - .delete_source(DeleteSourceRequest { - index_uid: index_uid.clone().into(), - source_id: source_id.to_string() - }) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Source { .. }) - )); - assert!(matches!( - metastore - .delete_source(DeleteSourceRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - source_id: source_id.to_string() - }) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - assert!(matches!( - metastore - .delete_source(DeleteSourceRequest { - index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), - source_id: source_id.to_string() - }) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_reset_checkpoint() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-reset-checkpoint"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let source_ids: Vec = (0..2).map(|i| format!("{index_id}--source-{i}")).collect(); - let split_ids: Vec = (0..2).map(|i| format!("{index_id}--split-{i}")).collect(); - - for (source_id, split_id) in source_ids.iter().zip(split_ids.iter()) { - let source = SourceConfig { - source_id: source_id.clone(), - max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), - desired_num_pipelines: NonZeroUsize::new(1).unwrap(), - enabled: true, - source_params: SourceParams::void(), - transform_config: None, - input_format: SourceInputFormat::Json, - }; - metastore - .add_source( - AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) - .unwrap(), - ) - .await - .unwrap(); - - let split_metadata = SplitMetadata { - split_id: split_id.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - } - assert!(!metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap() - .checkpoint - .is_empty()); - - metastore - .reset_source_checkpoint(ResetSourceCheckpointRequest { - index_uid: index_uid.clone().into(), - source_id: source_ids[0].clone(), - }) - .await - .unwrap(); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - assert!(index_metadata - .checkpoint - .source_checkpoint(&source_ids[0]) - .is_none()); - - assert!(index_metadata - .checkpoint - .source_checkpoint(&source_ids[1]) - .is_some()); - - assert!(matches!( - metastore - .reset_source_checkpoint(ResetSourceCheckpointRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - source_id: source_ids[1].clone(), - }) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - assert!(matches!( - metastore - .reset_source_checkpoint(ResetSourceCheckpointRequest { - index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), - source_id: source_ids[1].to_string(), - }) - .await - .unwrap_err(), - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - metastore - .reset_source_checkpoint(ResetSourceCheckpointRequest { - index_uid: index_uid.clone().into(), - source_id: source_ids[1].to_string(), - }) - .await - .unwrap(); - - assert!(metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap() - .checkpoint - .is_empty()); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_publish_splits_empty_splits_array_is_allowed< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-publish-splits-empty"); - let non_existent_index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - - let source_id = format!("{index_id}--source"); - - // Publish a split on a non-existent index - { - let publish_splits_request = PublishSplitsRequest { - index_uid: non_existent_index_uid.to_string(), - index_checkpoint_delta_json_opt: Some({ - let offsets = 1..10; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - } - - // Update the checkpoint, by publishing an empty array of splits with a non-empty - // checkpoint. This operation is allowed and used in the Indexer. - { - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - index_checkpoint_delta_json_opt: Some({ - let offsets = 0..100; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let source_checkpoint = index_metadata - .checkpoint - .source_checkpoint(&source_id) - .unwrap(); - assert_eq!(source_checkpoint.num_partitions(), 1); - assert_eq!( - source_checkpoint - .position_for_partition(&PartitionId::default()) - .unwrap(), - &Position::offset(100u64 - 1) - ); - cleanup_index(&mut metastore, index_uid).await; - } -} - -pub async fn test_metastore_publish_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - - let index_id = append_random_suffix("test-publish-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let source_id = format!("{index_id}--source"); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - time_range: Some(0..=99), - create_timestamp: current_timestamp, - ..Default::default() - }; - - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - time_range: Some(30..=99), - create_timestamp: current_timestamp, - ..Default::default() - }; - - // Publish a split on a non-existent index - { - let publish_splits_request = PublishSplitsRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - staged_split_ids: vec!["split-not-found".to_string()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 0..10; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - } - - // Publish a split on a wrong index uid - { - let publish_splits_request = PublishSplitsRequest { - index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), - staged_split_ids: vec!["split-not-found".to_string()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 0..10; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - } - - // Publish a non-existent split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec!["split-not-found".to_string()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a staged split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a published split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 1..12; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - println!("{:?}", error); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a non-staged split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 12..15; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 15..18; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a staged split and non-existent split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 15..18; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a published split and non-existent split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 15..18; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 18..24; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a non-staged split and non-existent split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 18..24; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 24..26; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish staged splits on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_2.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 24..26; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish a staged split and published split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 26..28; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 28..30; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Publish published splits on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 30..31; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let publish_splits_resquest = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 30..31; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_resquest) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::CheckpointDelta { .. }, - .. - } - )); - - cleanup_index(&mut metastore, index_uid).await; - } -} - -pub async fn test_metastore_publish_splits_concurrency< - MetastoreToTest: MetastoreService + DefaultForTest + Clone, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-publish-concurrency"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let source_id = format!("{index_id}--source"); - - let mut join_handles = Vec::with_capacity(10); - - for partition_id in 0..10 { - let mut metastore_clone = metastore.clone(); - let index_id = index_id.clone(); - let source_id = source_id.clone(); - - let join_handle = tokio::spawn({ - let index_uid = index_uid.clone(); - async move { - let split_id = format!("{index_id}--split-{partition_id}"); - let split_metadata = SplitMetadata { - split_id: split_id.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata.clone(), - ) - .unwrap(); - metastore_clone - .stage_splits(stage_splits_request) - .await - .unwrap(); - let source_delta = SourceCheckpointDelta::from_partition_delta( - PartitionId::from(partition_id as u64), - Position::Beginning, - Position::offset(partition_id as u64), - ) - .unwrap(); - let checkpoint_delta = IndexCheckpointDelta { - source_id, - source_delta, - }; - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id.clone()], - index_checkpoint_delta_json_opt: Some( - serde_json::to_string(&checkpoint_delta).unwrap(), - ), - ..Default::default() - }; - metastore_clone - .publish_splits(publish_splits_request) - .await - .unwrap(); - } - }); - join_handles.push(join_handle); - } - try_join_all(join_handles).await.unwrap(); - - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let source_checkpoint = index_metadata - .checkpoint - .source_checkpoint(&source_id) - .unwrap(); - - assert_eq!(source_checkpoint.num_partitions(), 10); - - cleanup_index(&mut metastore, index_uid).await -} - -pub async fn test_metastore_replace_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - - let index_id = append_random_suffix("test-replace-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - time_range: None, - create_timestamp: current_timestamp, - ..Default::default() - }; - - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - time_range: None, - create_timestamp: current_timestamp, - ..Default::default() - }; - - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - time_range: None, - create_timestamp: current_timestamp, - ..Default::default() - }; - - // Replace splits on a non-existent index - { - let publish_splits_request = PublishSplitsRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - staged_split_ids: vec!["split-not-found-1".to_string()], - replaced_split_ids: vec!["split-not-found-2".to_string()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - } - - // Replace a non-existent split on an index - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec!["split-not-found-1".to_string()], - replaced_split_ids: vec!["split-not-found-2".to_string()], - ..Default::default() - }; - // TODO source id - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Replace a publish split with a non existing split - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - // TODO Source id - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone()], - replaced_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Replace a publish split with a deleted split - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_2.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - // TODO source_id - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone()], - replaced_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Replace a publish split with mixed splits - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_2.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone(), split_id_3.clone()], - replaced_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) // TODO source id - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Splits { .. }) - )); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Replace a deleted split with a new split - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_2.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone()], - replaced_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - let error = metastore - .publish_splits(publish_splits_request) - .await - .unwrap_err(); - assert!( - matches!(error, MetastoreError::FailedPrecondition { entity: EntityKind::Splits { split_ids }, .. } if split_ids == [split_id_1.clone()]) - ); - - cleanup_index(&mut metastore, index_uid).await; - } - - // Replace a publish split with staged splits - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_split_metadata( - index_uid.clone(), - split_metadata_1.clone(), - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_2.clone(), split_metadata_3.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - // TODO Source id - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone(), split_id_3.clone()], - replaced_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - cleanup_index(&mut metastore, index_uid).await; - } -} - -pub async fn test_metastore_mark_splits_for_deletion< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - - let index_id = append_random_suffix("test-mark-splits-for-deletion"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new("index-not-found:0".into(), Vec::new()); - let error = metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec!["split-not-found".to_string()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_1.clone()) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_2.clone()) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_2.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_3.clone()) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_3.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - // Sleep for 1s so we can observe the timestamp update. - sleep(Duration::from_secs(1)).await; - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_3.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let list_splits_request = ListSplitsRequest::try_from_list_splits_query( - ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::MarkedForDeletion), - ) - .unwrap(); - let marked_splits = metastore - .list_splits(list_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - - assert_eq!(marked_splits.len(), 1); - assert_eq!(marked_splits[0].split_id(), split_id_3); - - let split_3_update_timestamp = marked_splits[0].update_timestamp; - assert!(current_timestamp < split_3_update_timestamp); - - // Sleep for 1s so we can observe the timestamp update. - sleep(Duration::from_secs(1)).await; - - let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( - index_uid.clone(), - vec![ - split_id_1.clone(), - split_id_2.clone(), - split_id_3.clone(), - "split-not-found".to_string(), - ], - ); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let list_splits_request = ListSplitsRequest::try_from_list_splits_query( - ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::MarkedForDeletion), - ) - .unwrap(); - let mut marked_splits = metastore - .list_splits(list_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - - marked_splits.sort_by_key(|split| split.split_id().to_string()); - - assert_eq!(marked_splits.len(), 3); - - assert_eq!(marked_splits[0].split_id(), split_id_1); - assert!(current_timestamp + 2 <= marked_splits[0].update_timestamp); - - assert_eq!(marked_splits[1].split_id(), split_id_2); - assert!(current_timestamp + 2 <= marked_splits[1].update_timestamp); - - assert_eq!(marked_splits[2].split_id(), split_id_3); - assert_eq!(marked_splits[2].update_timestamp, split_3_update_timestamp); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_delete_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-delete-splits"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let delete_splits_request = DeleteSplitsRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - split_ids: Vec::new(), - }; - let error = metastore - .delete_splits(delete_splits_request) - .await - .unwrap_err(); - - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let index_not_existing_uid = IndexUid::new_with_random_ulid(&index_id); - // Check error if index does not exist. - let delete_splits_request = DeleteSplitsRequest { - index_uid: index_not_existing_uid.to_string(), - split_ids: Vec::new(), - }; - let error = metastore - .delete_splits(delete_splits_request) - .await - .unwrap_err(); - - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let delete_splits_request = DeleteSplitsRequest { - index_uid: index_uid.clone().into(), - split_ids: vec!["split-not-found".to_string()], - }; - metastore - .delete_splits(delete_splits_request) - .await - .unwrap(); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_1.clone()) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let stage_splits_request = - StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_2.clone()) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let delete_splits_request = DeleteSplitsRequest { - index_uid: index_uid.clone().into(), - split_ids: vec![split_id_1.clone(), split_id_2.clone()], - }; - let error = metastore - .delete_splits(delete_splits_request) - .await - .unwrap_err(); - - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - )); - - assert_eq!( - metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap() - .len(), - 2 - ); - - let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( - index_uid.clone(), - vec![split_id_1.clone(), split_id_2.clone()], - ); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - - let delete_splits_request = DeleteSplitsRequest { - index_uid: index_uid.clone().into(), - split_ids: vec![ - split_id_1.clone(), - split_id_2.clone(), - "split-not-found".to_string(), - ], - }; - metastore - .delete_splits(delete_splits_request) - .await - .unwrap(); - - assert_eq!( - metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap() - .len(), - 0 - ); - - cleanup_index(&mut metastore, index_uid).await; -} - -pub async fn test_metastore_list_all_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let index_id = append_random_suffix("test-list-all-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let split_id_4 = format!("{index_id}--split-4"); - let split_metadata_4 = SplitMetadata { - split_id: split_id_4.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let split_id_5 = format!("{index_id}--split-5"); - let split_metadata_5 = SplitMetadata { - split_id: split_id_5.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - let split_id_6 = format!("{index_id}--split-6"); - let split_metadata_6 = SplitMetadata { - split_id: split_id_6.clone(), - index_uid: index_uid.clone(), - ..Default::default() - }; - - let error = metastore - .list_splits( - ListSplitsRequest::try_from_index_uid(IndexUid::new_with_random_ulid( - "index-not-found", - )) - .unwrap(), - ) - .await - .unwrap_err(); - assert!(matches!( - error, - // TODO: This discrepancy is tracked in #3760. - MetastoreError::NotFound(EntityKind::Index { .. } | EntityKind::Indexes { .. }) - )); - - let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![ - split_metadata_1, - split_metadata_2, - split_metadata_3, - split_metadata_4, - split_metadata_5, - split_metadata_6, - ], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().to_string(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let mark_splits_for_deletion = MarkSplitsForDeletionRequest::new( - index_uid.clone(), - vec![split_id_3.clone(), split_id_4.clone()], - ); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion) - .await - .unwrap(); - - let splits = metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5, - &split_id_6 - ] - ); - - cleanup_index(&mut metastore, index_uid.clone()).await; -} - -pub async fn test_metastore_list_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - - let index_id = append_random_suffix("test-list-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - time_range: Some(0..=99), - create_timestamp: current_timestamp, - maturity: SplitMaturity::Immature { - maturation_period: Duration::from_secs(0), - }, - tags: to_btree_set(&["tag!", "tag:foo", "$tag!", "$tag:bar"]), - delete_opstamp: 3, - ..Default::default() - }; - - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - time_range: Some(100..=199), - create_timestamp: current_timestamp, - maturity: SplitMaturity::Immature { - maturation_period: Duration::from_secs(10), - }, - tags: to_btree_set(&["tag!", "$tag!", "$tag:bar"]), - delete_opstamp: 1, - ..Default::default() - }; - - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - time_range: Some(200..=299), - create_timestamp: current_timestamp, - maturity: SplitMaturity::Immature { - maturation_period: Duration::from_secs(20), - }, - tags: to_btree_set(&["tag!", "tag:foo", "tag:baz", "$tag!"]), - delete_opstamp: 5, - ..Default::default() - }; - - let split_id_4 = format!("{index_id}--split-4"); - let split_metadata_4 = SplitMetadata { - split_id: split_id_4.clone(), - index_uid: index_uid.clone(), - time_range: Some(300..=399), - tags: to_btree_set(&["tag!", "tag:foo", "$tag!"]), - delete_opstamp: 7, - ..Default::default() - }; - - let split_id_5 = format!("{index_id}--split-5"); - let split_metadata_5 = SplitMetadata { - split_id: split_id_5.clone(), - index_uid: index_uid.clone(), - time_range: None, - create_timestamp: current_timestamp, - tags: to_btree_set(&["tag!", "tag:baz", "tag:biz", "$tag!"]), - delete_opstamp: 9, - ..Default::default() - }; - - { - let query = - ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let error = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap_err(); - assert!(matches!( - error, - // TODO: This discrepancy is tracked in #3760. - MetastoreError::NotFound(EntityKind::Index { .. } | EntityKind::Indexes { .. }) - )); - } - { - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - split_metadata_4.clone(), - split_metadata_5.clone(), - ], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let query = ListSplitsQuery::for_index(index_uid.clone()).with_limit(3); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!( - splits.len(), - 3, - "Expected number of splits returned to match limit.", - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()).with_offset(3); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!( - splits.len(), - 2, - "Expected 3 splits to be skipped out of the 5 provided splits.", - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(99); - - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids: Vec<&str> = splits - .iter() - .map(|split| split.split_id()) - .sorted() - .collect(); - assert_eq!(split_ids, &[&split_id_1, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(200); - - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_end_lt(200); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(100); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(101); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(199); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(200); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_2, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(201); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(299); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(300); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_1, &split_id_2, &split_id_3, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(0) - .with_time_range_end_lt(301); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5 - ] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(301) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(300) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(299) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(201) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(200) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_3, &split_id_4, &split_id_5]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(199) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(101) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(101) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(100) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - - assert_eq!( - split_ids, - &[&split_id_2, &split_id_3, &split_id_4, &split_id_5] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(99) - .with_time_range_end_lt(400); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5 - ] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_time_range_start_gte(1000) - .with_time_range_end_lt(1100); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_5]); - - // Artificially increase the create_timestamp - sleep(Duration::from_secs(1)).await; - // add a split without tag - let split_id_6 = format!("{index_id}--split-6"); - let split_metadata_6 = SplitMetadata { - split_id: split_id_6.clone(), - index_uid: index_uid.clone(), - time_range: None, - create_timestamp: OffsetDateTime::now_utc().unix_timestamp(), - ..Default::default() - }; - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_6.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let query = - ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5, - &split_id_6, - ] - ); - - let tag_filter_ast = TagFilterAst::Or(vec![ - TagFilterAst::Or(vec![no_tag("$tag!"), tag("$tag:bar")]), - TagFilterAst::Or(vec![no_tag("tag!"), tag("tag:baz")]), - ]); - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::Staged) - .with_tags_filter(tag_filter_ast); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_5, - &split_id_6, - ] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_update_timestamp_gte(current_timestamp); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5, - &split_id_6, - ] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_update_timestamp_gte(split_metadata_6.create_timestamp); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids: Vec<&String> = splits - .iter() - .map(|split| &split.split_metadata.split_id) - .sorted() - .collect(); - assert_eq!(split_ids, vec![&split_id_6]); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .with_create_timestamp_lt(split_metadata_6.create_timestamp); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[ - &split_id_1, - &split_id_2, - &split_id_3, - &split_id_4, - &split_id_5, - ] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()).with_delete_opstamp_lt(6); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_1, &split_id_2, &split_id_3, &split_id_6,] - ); - - // Test maturity filter - let maturity_evaluation_timestamp = - OffsetDateTime::from_unix_timestamp(current_timestamp).unwrap(); - let query = ListSplitsQuery::for_index(index_uid.clone()) - .retain_mature(maturity_evaluation_timestamp); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!( - split_ids, - &[&split_id_1, &split_id_4, &split_id_5, &split_id_6,] - ); - - let query = ListSplitsQuery::for_index(index_uid.clone()) - .retain_immature(maturity_evaluation_timestamp); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_2, &split_id_3]); - - cleanup_index(&mut metastore, index_uid).await; - } -} - -pub async fn test_metastore_split_update_timestamp< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - - let mut current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - - let index_id = append_random_suffix("split-update-timestamp"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); +use self::shard::RunTests; +use crate::metastore::MetastoreServiceStreamSplitsExt; +use crate::{ListSplitsRequestExt, MetastoreServiceExt, Split}; - let source_id = format!("{index_id}--source"); - - let split_id = format!("{index_id}--split"); - let split_metadata = SplitMetadata { - split_id: split_id.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - ..Default::default() - }; - - // Create an index - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - // wait for 1s, stage split & check `update_timestamp` - sleep(Duration::from_secs(1)).await; - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - sleep(Duration::from_secs(1)).await; - let split_meta = metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap()[0] - .clone(); - assert!(split_meta.update_timestamp > current_timestamp); - assert!(split_meta.publish_timestamp.is_none()); - - current_timestamp = split_meta.update_timestamp; - - // wait for 1s, publish split & check `update_timestamp` - sleep(Duration::from_secs(1)).await; - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id.clone()], - index_checkpoint_delta_json_opt: Some({ - let offsets = 0..5; - let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); - serde_json::to_string(&checkpoint_delta).unwrap() - }), - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - let split_meta = metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap()[0] - .clone(); - assert!(split_meta.update_timestamp > current_timestamp); - assert_eq!( - split_meta.publish_timestamp, - Some(split_meta.update_timestamp) - ); - current_timestamp = split_meta.update_timestamp; - - // wait for 1s, mark split for deletion & check `update_timestamp` - sleep(Duration::from_secs(1)).await; - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id.clone()]); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await - .unwrap(); - let split_meta = metastore - .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap()[0] - .clone(); - assert!(split_meta.update_timestamp > current_timestamp); - assert!(split_meta.publish_timestamp.is_some()); - - cleanup_index(&mut metastore, index_uid).await; +#[async_trait] +pub trait DefaultForTest { + async fn default_for_test() -> Self; } -pub async fn test_metastore_create_delete_task< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let index_id = append_random_suffix("add-delete-task"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - let delete_query = DeleteQuery { - index_uid: index_uid.clone().into(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; - - // Create a delete task on non-existing index. - let error = metastore - .create_delete_task(DeleteQuery { - index_uid: IndexUid::new_with_random_ulid("does-not-exist").to_string(), - ..delete_query.clone() - }) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - // Create a delete task on an index with wrong incarnation_id - let error = metastore - .create_delete_task(DeleteQuery { - index_uid: IndexUid::from_parts(&index_id, "12345").to_string(), - ..delete_query.clone() - }) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - // Create a delete task. - let delete_task_1 = metastore - .create_delete_task(delete_query.clone()) - .await - .unwrap(); - assert!(delete_task_1.opstamp > 0); - let delete_query_1 = delete_task_1.delete_query.unwrap(); - assert_eq!(delete_query_1.index_uid, delete_query.index_uid); - assert_eq!(delete_query_1.start_timestamp, delete_query.start_timestamp); - assert_eq!(delete_query_1.end_timestamp, delete_query.end_timestamp); - let delete_task_2 = metastore - .create_delete_task(delete_query.clone()) - .await - .unwrap(); - assert!(delete_task_2.opstamp > delete_task_1.opstamp); +// We implement the trait to test the gRPC adapter backed by a file backed metastore. +#[async_trait] +impl DefaultForTest for MetastoreServiceGrpcClientAdapter> { + async fn default_for_test() -> Self { + use quickwit_proto::tonic::transport::Server; + use quickwit_storage::RamStorage; + + use crate::FileBackedMetastore; + let metastore = + FileBackedMetastore::try_new(std::sync::Arc::new(RamStorage::default()), None) + .await + .unwrap(); + let (client, server) = tokio::io::duplex(1024); + tokio::spawn(async move { + Server::builder() + .add_service(MetastoreServiceClient::new(metastore).as_grpc_service()) + .serve_with_incoming(futures::stream::iter(vec![Ok::<_, std::io::Error>(server)])) + .await + }); + let channel = create_channel(client).await.unwrap(); + let (_, connection_keys_watcher) = + tokio::sync::watch::channel(std::collections::HashSet::new()); - cleanup_index(&mut metastore, index_uid).await; + MetastoreServiceGrpcClientAdapter::new( + MetastoreServiceGrpcClient::new(channel), + connection_keys_watcher, + ) + } } -pub async fn test_metastore_last_delete_opstamp< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let index_id_1 = append_random_suffix("test-last-delete-opstamp-1"); - let index_uri_1 = format!("ram:///indexes/{index_id_1}"); - let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); - let index_id_2 = append_random_suffix("test-last-delete-opstamp-2"); - let index_uri_2 = format!("ram:///indexes/{index_id_2}"); - let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); - let index_uid_1: IndexUid = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_2: IndexUid = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap()) - .await - .unwrap() - .index_uid - .into(); - - let delete_query_index_1 = DeleteQuery { - index_uid: index_uid_1.to_string(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; - let delete_query_index_2 = DeleteQuery { - index_uid: index_uid_2.to_string(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; +impl MetastoreServiceExt + for MetastoreServiceGrpcClientAdapter> +{ +} - let last_opstamp_index_1_with_no_task = metastore - .last_delete_opstamp(LastDeleteOpstampRequest { - index_uid: index_uid_1.to_string(), - }) - .await - .unwrap() - .last_delete_opstamp; - assert_eq!(last_opstamp_index_1_with_no_task, 0); +impl RunTests for MetastoreServiceGrpcClientAdapter> {} - // Create a delete task. - metastore - .create_delete_task(delete_query_index_1.clone()) - .await - .unwrap(); - let delete_task_2 = metastore - .create_delete_task(delete_query_index_1.clone()) - .await - .unwrap(); - let delete_task_3 = metastore - .create_delete_task(delete_query_index_2.clone()) - .await - .unwrap(); +async fn create_channel(client: tokio::io::DuplexStream) -> anyhow::Result { + use http::Uri; + use quickwit_proto::tonic::transport::Endpoint; - let last_opstamp_index_1 = metastore - .last_delete_opstamp(LastDeleteOpstampRequest { - index_uid: index_uid_1.to_string(), - }) - .await - .unwrap() - .last_delete_opstamp; - let last_opstamp_index_2 = metastore - .last_delete_opstamp(LastDeleteOpstampRequest { - index_uid: index_uid_2.to_string(), - }) - .await - .unwrap() - .last_delete_opstamp; - assert_eq!(last_opstamp_index_1, delete_task_2.opstamp); - assert_eq!(last_opstamp_index_2, delete_task_3.opstamp); - cleanup_index(&mut metastore, index_uid_1).await; - cleanup_index(&mut metastore, index_uid_2).await; + let mut client = Some(client); + let channel = Endpoint::try_from("http://test.server")? + .connect_with_connector(tower::service_fn(move |_: Uri| { + let client = client.take(); + async move { + client.ok_or_else(|| { + std::io::Error::new(std::io::ErrorKind::Other, "client already taken") + }) + } + })) + .await?; + Ok(channel) } -pub async fn test_metastore_delete_index_with_tasks< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let index_id = append_random_suffix("delete-delete-tasks"); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - let delete_query = DeleteQuery { - index_uid: index_uid.clone().into(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; - let _ = metastore - .create_delete_task(delete_query.clone()) - .await - .unwrap(); - let _ = metastore - .create_delete_task(delete_query.clone()) - .await - .unwrap(); +crate::metastore_test_suite!( + quickwit_proto::metastore::MetastoreServiceGrpcClientAdapter< + quickwit_proto::metastore::metastore_service_grpc_client::MetastoreServiceGrpcClient< + quickwit_proto::tonic::transport::Channel, + >, + > +); - metastore - .delete_index(DeleteIndexRequest { - index_uid: index_uid.clone().into(), - }) - .await - .unwrap(); +fn collect_split_ids(splits: &[Split]) -> Vec<&str> { + splits + .iter() + .map(|split| split.split_id()) + .sorted() + .collect() } -pub async fn test_metastore_list_delete_tasks< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let index_id_1 = append_random_suffix("test-list-delete-tasks-1"); - let index_uri_1 = format!("ram:///indexes/{index_id_1}"); - let index_config_1 = IndexConfig::for_test(&index_id_1, &index_uri_1); - let index_id_2 = append_random_suffix("test-list-delete-tasks-2"); - let index_uri_2 = format!("ram:///indexes/{index_id_2}"); - let index_config_2 = IndexConfig::for_test(&index_id_2, &index_uri_2); - let index_uid_1: IndexUid = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let index_uid_2: IndexUid = metastore - .create_index(CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap()) - .await - .unwrap() - .index_uid - .into(); - let delete_query_index_1 = DeleteQuery { - index_uid: index_uid_1.to_string(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; - let delete_query_index_2 = DeleteQuery { - index_uid: index_uid_2.to_string(), - query_ast: qast_json_helper("my_field:my_value", &[]), - start_timestamp: Some(1), - end_timestamp: Some(2), - }; - - // Create a delete task. - let delete_task_1 = metastore - .create_delete_task(delete_query_index_1.clone()) - .await - .unwrap(); - let delete_task_2 = metastore - .create_delete_task(delete_query_index_1.clone()) - .await - .unwrap(); - let _ = metastore - .create_delete_task(delete_query_index_2.clone()) - .await - .unwrap(); - - let all_index_id_1_delete_tasks = metastore - .list_delete_tasks(ListDeleteTasksRequest::new(index_uid_1.clone(), 0)) - .await - .unwrap() - .delete_tasks; - assert_eq!(all_index_id_1_delete_tasks.len(), 2); +fn to_btree_set(tags: &[&str]) -> BTreeSet { + tags.iter().map(|tag| tag.to_string()).collect() +} - let recent_index_id_1_delete_tasks = metastore - .list_delete_tasks(ListDeleteTasksRequest::new( - index_uid_1.clone(), - delete_task_1.opstamp, - )) +async fn cleanup_index(metastore: &mut dyn MetastoreServiceExt, index_uid: IndexUid) { + // List all splits. + let all_splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() - .delete_tasks; - assert_eq!(recent_index_id_1_delete_tasks.len(), 1); - assert_eq!( - recent_index_id_1_delete_tasks[0].opstamp, - delete_task_2.opstamp - ); - cleanup_index(&mut metastore, index_uid_1).await; - cleanup_index(&mut metastore, index_uid_2).await; -} - -pub async fn test_metastore_list_stale_splits< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - let index_id = append_random_suffix("test-list-stale-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 20, - ..Default::default() - }; - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 10, - ..Default::default() - }; - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 0, - ..Default::default() - }; - let split_id_4 = format!("{index_id}--split-4"); - let split_metadata_4 = SplitMetadata { - split_id: split_id_4.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 20, - ..Default::default() - }; - // immature split - let split_id_5 = format!("{index_id}--split-5"); - let split_metadata_5 = SplitMetadata { - split_id: split_id_5.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - maturity: SplitMaturity::Immature { - maturation_period: Duration::from_secs(100), - }, - delete_opstamp: 0, - ..Default::default() - }; - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - delete_opstamp: 0, - num_splits: 100, - }; - let error = metastore - .list_stale_splits(list_stale_splits_request) + .collect_splits() .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - { - info!("list stale splits on an index"); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - split_metadata_5.clone(), - ], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - // Sleep for 1 second to have different publish timestamps. - sleep(Duration::from_secs(1)).await; - - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_4.clone()], - ) .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id_4.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - // Sleep for 1 second to have different publish timestamps. - tokio::time::sleep(Duration::from_secs(1)).await; - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone(), split_id_5.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 100, - num_splits: 1, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!(splits.len(), 1); - assert_eq!( - splits[0].split_metadata.delete_opstamp, - split_metadata_2.delete_opstamp - ); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 100, - num_splits: 4, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!(splits.len(), 3); - assert_eq!(splits[0].split_id(), split_metadata_2.split_id()); - assert_eq!(splits[1].split_id(), split_metadata_4.split_id()); - assert_eq!(splits[2].split_id(), split_metadata_1.split_id()); - assert_eq!( - splits[2].split_metadata.delete_opstamp, - split_metadata_1.delete_opstamp - ); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 20, - num_splits: 2, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!(splits.len(), 1); - assert_eq!( - splits[0].split_metadata.delete_opstamp, - split_metadata_2.delete_opstamp - ); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 10, - num_splits: 2, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert!(splits.is_empty()); - cleanup_index(&mut metastore, index_uid).await; - } -} - -pub async fn test_metastore_update_splits_delete_opstamp< - MetastoreToTest: MetastoreService + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - let index_id = append_random_suffix("update-splits-delete-opstamp"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 20, - ..Default::default() - }; - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 10, - ..Default::default() - }; - let split_id_3 = format!("{index_id}--split-3"); - let split_metadata_3 = SplitMetadata { - split_id: split_id_3.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 0, - ..Default::default() - }; - - { - info!("update splits delete opstamp on a non-existent index"); - let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { - index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), - split_ids: vec![split_id_1.clone()], - delete_opstamp: 10, - }; - let metastore_err = metastore - .update_splits_delete_opstamp(update_splits_delete_opstamp_request) - .await - .unwrap_err(); - error!(err=?metastore_err); - assert!(matches!( - metastore_err, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - } - { - info!("update splits delete opstamp on an index"); - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); + if !all_splits.is_empty() { + let all_split_ids: Vec = all_splits + .iter() + .map(|split| split.split_id().to_string()) + .collect(); - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - ], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - ..Default::default() - }; + // Mark splits for deletion. + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), all_split_ids.clone()); metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 100, - num_splits: 2, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await - .unwrap() - .deserialize_splits() .unwrap(); - assert_eq!(splits.len(), 2); - let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { + // Delete splits. + let delete_splits_request = DeleteSplitsRequest { index_uid: index_uid.clone().into(), - split_ids: vec![split_id_1.clone(), split_id_2.clone()], - delete_opstamp: 100, + split_ids: all_split_ids, }; metastore - .update_splits_delete_opstamp(update_splits_delete_opstamp_request) - .await - .unwrap(); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 100, - num_splits: 2, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - assert_eq!(splits.len(), 0); - - let list_stale_splits_request = ListStaleSplitsRequest { - index_uid: index_uid.clone().into(), - delete_opstamp: 200, - num_splits: 2, - }; - let splits = metastore - .list_stale_splits(list_stale_splits_request) + .delete_splits(delete_splits_request) .await - .unwrap() - .deserialize_splits() .unwrap(); - assert_eq!(splits.len(), 2); - assert_eq!(splits[0].split_metadata.delete_opstamp, 100); - assert_eq!(splits[1].split_metadata.delete_opstamp, 100); - - cleanup_index(&mut metastore, index_uid).await; } -} - -pub async fn test_metastore_stage_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; - let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - let index_id = append_random_suffix("test-stage-splits"); - let index_uid = IndexUid::new_with_random_ulid(&index_id); - let index_uri = format!("ram:///indexes/{index_id}"); - let index_config = IndexConfig::for_test(&index_id, &index_uri); - - let split_id_1 = format!("{index_id}--split-1"); - let split_metadata_1 = SplitMetadata { - split_id: split_id_1.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 20, - ..Default::default() - }; - let split_id_2 = format!("{index_id}--split-2"); - let split_metadata_2 = SplitMetadata { - split_id: split_id_2.clone(), - index_uid: index_uid.clone(), - create_timestamp: current_timestamp, - delete_opstamp: 10, - ..Default::default() - }; - - // Stage a splits on a non-existent index - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - IndexUid::new_with_random_ulid("index-not-found"), - vec![split_metadata_1.clone()], - ) - .unwrap(); - let error = metastore - .stage_splits(stage_splits_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::NotFound(EntityKind::Index { .. }) - )); - - let create_index_request = - CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); - let index_uid: IndexUid = metastore - .create_index(create_index_request) - .await - .unwrap() - .index_uid - .into(); - - // Stage a split on an index - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) - .unwrap(); - metastore.stage_splits(stage_splits_request).await.unwrap(); - - let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let splits = metastore - .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) - .await - .unwrap() - .deserialize_splits() - .unwrap(); - let split_ids = collect_split_ids(&splits); - assert_eq!(split_ids, &[&split_id_1, &split_id_2]); - - // Stage a existent-staged-split on an index - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone()], - ) - .unwrap(); - metastore - .stage_splits(stage_splits_request) - .await - .expect("Pre-existing staged splits should be updated."); - - let publish_splits_request = PublishSplitsRequest { - index_uid: index_uid.clone().into(), - staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], - ..Default::default() - }; - metastore - .publish_splits(publish_splits_request) - .await - .unwrap(); - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_1.clone()], - ) - .unwrap(); - let error = metastore - .stage_splits(stage_splits_request) - .await - .expect_err("Metastore should not allow splits which are not `Staged` to be overwritten."); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - ),); - - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_2.clone()]); + // Delete index. metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) + .delete_index(DeleteIndexRequest { + index_uid: index_uid.clone().into(), + }) .await .unwrap(); - let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( - index_uid.clone(), - vec![split_metadata_2.clone()], - ) - .unwrap(); - let error = metastore - .stage_splits(stage_splits_request) - .await - .expect_err("Metastore should not allow splits which are not `Staged` to be overwritten."); - assert!(matches!( - error, - MetastoreError::FailedPrecondition { - entity: EntityKind::Splits { .. }, - .. - } - ),); - - cleanup_index(&mut metastore, index_uid).await; } #[macro_export] @@ -3865,174 +180,187 @@ macro_rules! metastore_test_suite { #[tokio::test] async fn test_metastore_create_index() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_create_index::<$metastore_type>().await; + $crate::tests::index::test_metastore_create_index::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_create_index_with_maximum_length() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_create_index_with_maximum_length::<$metastore_type>() - .await; + $crate::tests::index::test_metastore_create_index_with_maximum_length::< + $metastore_type, + >() + .await; } #[tokio::test] async fn test_metastore_index_exists() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_index_exists::<$metastore_type>().await; + $crate::tests::index::test_metastore_index_exists::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_index_metadata() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_index_metadata::<$metastore_type>().await; + $crate::tests::index::test_metastore_index_metadata::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_indexes() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_indexes::<$metastore_type>().await; + $crate::tests::index::test_metastore_list_indexes::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_all_indexes() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_all_indexes::<$metastore_type>().await; + $crate::tests::index::test_metastore_list_all_indexes::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_delete_index() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_delete_index::<$metastore_type>().await; + $crate::tests::index::test_metastore_delete_index::<$metastore_type>().await; } // Split API tests // // - stage_splits // - publish_splits + // - stream_splits // - mark_splits_for_deletion // - delete_splits #[tokio::test] async fn test_metastore_publish_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_publish_splits::<$metastore_type>().await; + $crate::tests::split::test_metastore_publish_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_publish_splits_concurrency() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_publish_splits_concurrency::<$metastore_type>().await; + $crate::tests::split::test_metastore_publish_splits_concurrency::<$metastore_type>( + ) + .await; } #[tokio::test] async fn test_metastore_publish_splits_empty_splits_array_is_allowed() { - $crate::tests::test_metastore_publish_splits_empty_splits_array_is_allowed::< - $metastore_type, - >() - .await; + $crate::tests::split::test_metastore_publish_splits_empty_splits_array_is_allowed::< + $metastore_type, + >() + .await; } #[tokio::test] async fn test_metastore_replace_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_replace_splits::<$metastore_type>().await; + $crate::tests::split::test_metastore_replace_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_mark_splits_for_deletion() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_mark_splits_for_deletion::<$metastore_type>().await; + $crate::tests::split::test_metastore_mark_splits_for_deletion::<$metastore_type>() + .await; } #[tokio::test] async fn test_metastore_delete_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_delete_splits::<$metastore_type>().await; + $crate::tests::split::test_metastore_delete_splits::<$metastore_type>().await; + } + + #[tokio::test] + async fn test_metastore_stream_splits() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::list_splits::test_metastore_stream_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_all_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_all_splits::<$metastore_type>().await; + $crate::tests::list_splits::test_metastore_list_all_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_splits::<$metastore_type>().await; + $crate::tests::list_splits::test_metastore_list_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_split_update_timestamp() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_split_update_timestamp::<$metastore_type>().await; + $crate::tests::split::test_metastore_split_update_timestamp::<$metastore_type>() + .await; } #[tokio::test] async fn test_metastore_add_source() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_add_source::<$metastore_type>().await; + $crate::tests::source::test_metastore_add_source::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_toggle_source() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_toggle_source::<$metastore_type>().await; + $crate::tests::source::test_metastore_toggle_source::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_delete_source() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_delete_source::<$metastore_type>().await; + $crate::tests::source::test_metastore_delete_source::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_reset_checkpoint() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_reset_checkpoint::<$metastore_type>().await; + $crate::tests::source::test_metastore_reset_checkpoint::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_create_delete_task() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_create_delete_task::<$metastore_type>().await; + $crate::tests::delete_task::test_metastore_create_delete_task::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_last_delete_opstamp() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_last_delete_opstamp::<$metastore_type>().await; + $crate::tests::delete_task::test_metastore_last_delete_opstamp::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_delete_index_with_tasks() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_delete_index_with_tasks::<$metastore_type>().await; + $crate::tests::delete_task::test_metastore_delete_index_with_tasks::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_delete_tasks() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_delete_tasks::<$metastore_type>().await; + $crate::tests::delete_task::test_metastore_list_delete_tasks::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_list_stale_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_list_stale_splits::<$metastore_type>().await; + $crate::tests::list_splits::test_metastore_list_stale_splits::<$metastore_type>().await; } #[tokio::test] async fn test_metastore_update_splits_delete_opstamp() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_update_splits_delete_opstamp::<$metastore_type>() + $crate::tests::split::test_metastore_update_splits_delete_opstamp::<$metastore_type>() .await; } #[tokio::test] async fn test_metastore_stage_splits() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::test_metastore_stage_splits::<$metastore_type>().await; + $crate::tests::split::test_metastore_stage_splits::<$metastore_type>().await; } /// Shard API tests diff --git a/quickwit/quickwit-metastore/src/tests/source.rs b/quickwit/quickwit-metastore/src/tests/source.rs new file mode 100644 index 0000000000..3a5d223025 --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/source.rs @@ -0,0 +1,460 @@ +// 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::num::NonZeroUsize; + +use quickwit_common::rand::append_random_suffix; +use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, DeleteSourceRequest, EntityKind, IndexMetadataRequest, + MetastoreError, PublishSplitsRequest, ResetSourceCheckpointRequest, SourceType, + StageSplitsRequest, ToggleSourceRequest, +}; +use quickwit_proto::types::IndexUid; + +use super::DefaultForTest; +use crate::checkpoint::SourceCheckpoint; +use crate::tests::cleanup_index; +use crate::{ + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, MetastoreServiceExt, + SplitMetadata, StageSplitsRequestExt, +}; + +pub async fn test_metastore_add_source() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-add-source"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let source_id = format!("{index_id}--source"); + + let source = SourceConfig { + source_id: source_id.to_string(), + max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), + desired_num_pipelines: NonZeroUsize::new(1).unwrap(), + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + + assert_eq!( + metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap() + .checkpoint + .source_checkpoint(&source_id), + None + ); + + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + + let sources = &index_metadata.sources; + assert_eq!(sources.len(), 1); + assert!(sources.contains_key(&source_id)); + assert_eq!(sources.get(&source_id).unwrap().source_id, source_id); + assert_eq!( + sources.get(&source_id).unwrap().source_type(), + SourceType::Void + ); + assert_eq!( + index_metadata.checkpoint.source_checkpoint(&source_id), + Some(&SourceCheckpoint::default()) + ); + + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::AlreadyExists(EntityKind::Source { .. }) + )); + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new_with_random_ulid("index-not-found"), + source.clone() + ) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new_with_random_ulid(&index_id), + source + ) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_toggle_source() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-toggle-source"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let source_id = format!("{index_id}--source"); + let source = SourceConfig { + source_id: source_id.to_string(), + max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), + desired_num_pipelines: NonZeroUsize::new(1).unwrap(), + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let source = index_metadata.sources.get(&source_id).unwrap(); + assert_eq!(source.enabled, true); + + // Disable source. + metastore + .toggle_source(ToggleSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source.source_id.clone(), + enable: false, + }) + .await + .unwrap(); + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let source = index_metadata.sources.get(&source_id).unwrap(); + assert_eq!(source.enabled, false); + + // Enable source. + metastore + .toggle_source(ToggleSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source.source_id.clone(), + enable: true, + }) + .await + .unwrap(); + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let source = index_metadata.sources.get(&source_id).unwrap(); + assert_eq!(source.enabled, true); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_delete_source() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-delete-source"); + let index_uri = format!("ram:///indexes/{index_id}"); + let source_id = format!("{index_id}--source"); + + let source = SourceConfig { + source_id: source_id.to_string(), + max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), + desired_num_pipelines: NonZeroUsize::new(1).unwrap(), + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + + let index_config = IndexConfig::for_test(&index_id, index_uri.as_str()); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new_with_random_ulid("index-not-found"), + source.clone() + ) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new_with_random_ulid(&index_id), + source.clone() + ) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + metastore + .add_source( + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()).unwrap(), + ) + .await + .unwrap(); + metastore + .delete_source(DeleteSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + }) + .await + .unwrap(); + + let sources = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap() + .sources; + assert!(sources.is_empty()); + + assert!(matches!( + metastore + .delete_source(DeleteSourceRequest { + index_uid: index_uid.clone().into(), + source_id: source_id.to_string() + }) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Source { .. }) + )); + assert!(matches!( + metastore + .delete_source(DeleteSourceRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + source_id: source_id.to_string() + }) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + assert!(matches!( + metastore + .delete_source(DeleteSourceRequest { + index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), + source_id: source_id.to_string() + }) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_reset_checkpoint< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-reset-checkpoint"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let source_ids: Vec = (0..2).map(|i| format!("{index_id}--source-{i}")).collect(); + let split_ids: Vec = (0..2).map(|i| format!("{index_id}--split-{i}")).collect(); + + for (source_id, split_id) in source_ids.iter().zip(split_ids.iter()) { + let source = SourceConfig { + source_id: source_id.clone(), + max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(), + desired_num_pipelines: NonZeroUsize::new(1).unwrap(), + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + metastore + .add_source( + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) + .unwrap(), + ) + .await + .unwrap(); + + let split_metadata = SplitMetadata { + split_id: split_id.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + } + assert!(!metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap() + .checkpoint + .is_empty()); + + metastore + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: index_uid.clone().into(), + source_id: source_ids[0].clone(), + }) + .await + .unwrap(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert!(index_metadata + .checkpoint + .source_checkpoint(&source_ids[0]) + .is_none()); + + assert!(index_metadata + .checkpoint + .source_checkpoint(&source_ids[1]) + .is_some()); + + assert!(matches!( + metastore + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + source_id: source_ids[1].clone(), + }) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + assert!(matches!( + metastore + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), + source_id: source_ids[1].to_string(), + }) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + metastore + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: index_uid.clone().into(), + source_id: source_ids[1].to_string(), + }) + .await + .unwrap(); + + assert!(metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap() + .checkpoint + .is_empty()); + + cleanup_index(&mut metastore, index_uid).await; +} diff --git a/quickwit/quickwit-metastore/src/tests/split.rs b/quickwit/quickwit-metastore/src/tests/split.rs new file mode 100644 index 0000000000..09454b246f --- /dev/null +++ b/quickwit/quickwit-metastore/src/tests/split.rs @@ -0,0 +1,1831 @@ +// 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::time::Duration; + +use futures::future::try_join_all; +use quickwit_common::rand::append_random_suffix; +use quickwit_config::IndexConfig; +use quickwit_proto::metastore::{ + CreateIndexRequest, DeleteSplitsRequest, EntityKind, IndexMetadataRequest, ListSplitsRequest, + ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, PublishSplitsRequest, + StageSplitsRequest, UpdateSplitsDeleteOpstampRequest, +}; +use quickwit_proto::types::{IndexUid, Position}; +use time::OffsetDateTime; +use tokio::time::sleep; +use tracing::{error, info}; + +use super::DefaultForTest; +use crate::checkpoint::{IndexCheckpointDelta, PartitionId, SourceCheckpointDelta}; +use crate::metastore::MetastoreServiceStreamSplitsExt; +use crate::tests::{cleanup_index, collect_split_ids}; +use crate::{ + CreateIndexRequestExt, IndexMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, + ListSplitsResponseExt, MetastoreServiceExt, SplitMetadata, SplitState, StageSplitsRequestExt, +}; + +pub async fn test_metastore_publish_splits_empty_splits_array_is_allowed< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-publish-splits-empty"); + let non_existent_index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + + let source_id = format!("{index_id}--source"); + + // Publish a split on a non-existent index + { + let publish_splits_request = PublishSplitsRequest { + index_uid: non_existent_index_uid.to_string(), + index_checkpoint_delta_json_opt: Some({ + let offsets = 1..10; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + } + + // Update the checkpoint, by publishing an empty array of splits with a non-empty + // checkpoint. This operation is allowed and used in the Indexer. + { + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + index_checkpoint_delta_json_opt: Some({ + let offsets = 0..100; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let source_checkpoint = index_metadata + .checkpoint + .source_checkpoint(&source_id) + .unwrap(); + assert_eq!(source_checkpoint.num_partitions(), 1); + assert_eq!( + source_checkpoint + .position_for_partition(&PartitionId::default()) + .unwrap(), + &Position::offset(100u64 - 1) + ); + cleanup_index(&mut metastore, index_uid).await; + } +} + +pub async fn test_metastore_publish_splits< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + + let index_id = append_random_suffix("test-publish-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let source_id = format!("{index_id}--source"); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + time_range: Some(0..=99), + create_timestamp: current_timestamp, + ..Default::default() + }; + + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + time_range: Some(30..=99), + create_timestamp: current_timestamp, + ..Default::default() + }; + + // Publish a split on a non-existent index + { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + staged_split_ids: vec!["split-not-found".to_string()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 0..10; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + } + + // Publish a split on a wrong index uid + { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new_with_random_ulid(&index_id).to_string(), + staged_split_ids: vec!["split-not-found".to_string()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 0..10; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + } + + // Publish a non-existent split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec!["split-not-found".to_string()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a staged split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a published split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 1..12; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + println!("{:?}", error); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a non-staged split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 12..15; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 15..18; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a staged split and non-existent split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 15..18; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a published split and non-existent split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 15..18; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 18..24; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a non-staged split and non-existent split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 18..24; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), "split-not-found".to_string()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 24..26; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish staged splits on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_2.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 24..26; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish a staged split and published split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone(), split_metadata_2.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 26..28; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 28..30; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Publish published splits on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone(), split_metadata_2.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 30..31; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let publish_splits_resquest = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 30..31; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_resquest) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::CheckpointDelta { .. }, + .. + } + )); + + cleanup_index(&mut metastore, index_uid).await; + } +} + +pub async fn test_metastore_publish_splits_concurrency< + MetastoreToTest: MetastoreServiceExt + DefaultForTest + Clone, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-publish-concurrency"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let source_id = format!("{index_id}--source"); + + let mut join_handles = Vec::with_capacity(10); + + for partition_id in 0..10 { + let mut metastore_clone = metastore.clone(); + let index_id = index_id.clone(); + let source_id = source_id.clone(); + + let join_handle = tokio::spawn({ + let index_uid = index_uid.clone(); + async move { + let split_id = format!("{index_id}--split-{partition_id}"); + let split_metadata = SplitMetadata { + split_id: split_id.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata.clone(), + ) + .unwrap(); + metastore_clone + .stage_splits(stage_splits_request) + .await + .unwrap(); + let source_delta = SourceCheckpointDelta::from_partition_delta( + PartitionId::from(partition_id as u64), + Position::Beginning, + Position::offset(partition_id as u64), + ) + .unwrap(); + let checkpoint_delta = IndexCheckpointDelta { + source_id, + source_delta, + }; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id.clone()], + index_checkpoint_delta_json_opt: Some( + serde_json::to_string(&checkpoint_delta).unwrap(), + ), + ..Default::default() + }; + metastore_clone + .publish_splits(publish_splits_request) + .await + .unwrap(); + } + }); + join_handles.push(join_handle); + } + try_join_all(join_handles).await.unwrap(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let source_checkpoint = index_metadata + .checkpoint + .source_checkpoint(&source_id) + .unwrap(); + + assert_eq!(source_checkpoint.num_partitions(), 10); + + cleanup_index(&mut metastore, index_uid).await +} + +pub async fn test_metastore_replace_splits< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + + let index_id = append_random_suffix("test-replace-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + time_range: None, + create_timestamp: current_timestamp, + ..Default::default() + }; + + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + time_range: None, + create_timestamp: current_timestamp, + ..Default::default() + }; + + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + time_range: None, + create_timestamp: current_timestamp, + ..Default::default() + }; + + // Replace splits on a non-existent index + { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + staged_split_ids: vec!["split-not-found-1".to_string()], + replaced_split_ids: vec!["split-not-found-2".to_string()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + } + + // Replace a non-existent split on an index + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec!["split-not-found-1".to_string()], + replaced_split_ids: vec!["split-not-found-2".to_string()], + ..Default::default() + }; + // TODO source id + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Replace a publish split with a non existing split + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + // TODO Source id + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone()], + replaced_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Replace a publish split with a deleted split + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone(), split_metadata_2.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_2.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + // TODO source_id + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone()], + replaced_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Replace a publish split with mixed splits + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_2.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone(), split_id_3.clone()], + replaced_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) // TODO source id + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Splits { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Replace a deleted split with a new split + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_1.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_2.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone()], + replaced_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + let error = metastore + .publish_splits(publish_splits_request) + .await + .unwrap_err(); + assert!( + matches!(error, MetastoreError::FailedPrecondition { entity: EntityKind::Splits { split_ids }, .. } if split_ids == [split_id_1.clone()]) + ); + + cleanup_index(&mut metastore, index_uid).await; + } + + // Replace a publish split with staged splits + { + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata_1.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_2.clone(), split_metadata_3.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + // TODO Source id + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone(), split_id_3.clone()], + replaced_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + cleanup_index(&mut metastore, index_uid).await; + } +} + +pub async fn test_metastore_mark_splits_for_deletion< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + + let index_id = append_random_suffix("test-mark-splits-for-deletion"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new("index-not-found:0".into(), Vec::new()); + let error = metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec!["split-not-found".to_string()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_1.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_2.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_2.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_3.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_3.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + // Sleep for 1s so we can observe the timestamp update. + sleep(Duration::from_secs(1)).await; + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_3.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let list_splits_request = ListSplitsRequest::try_from_list_splits_query( + ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::MarkedForDeletion), + ) + .unwrap(); + let marked_splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + + assert_eq!(marked_splits.len(), 1); + assert_eq!(marked_splits[0].split_id(), split_id_3); + + let split_3_update_timestamp = marked_splits[0].update_timestamp; + assert!(current_timestamp < split_3_update_timestamp); + + // Sleep for 1s so we can observe the timestamp update. + sleep(Duration::from_secs(1)).await; + + let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( + index_uid.clone(), + vec![ + split_id_1.clone(), + split_id_2.clone(), + split_id_3.clone(), + "split-not-found".to_string(), + ], + ); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let list_splits_request = ListSplitsRequest::try_from_list_splits_query( + ListSplitsQuery::for_index(index_uid.clone()) + .with_split_state(SplitState::MarkedForDeletion), + ) + .unwrap(); + let mut marked_splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + + marked_splits.sort_by_key(|split| split.split_id().to_string()); + + assert_eq!(marked_splits.len(), 3); + + assert_eq!(marked_splits[0].split_id(), split_id_1); + assert!(current_timestamp + 2 <= marked_splits[0].update_timestamp); + + assert_eq!(marked_splits[1].split_id(), split_id_2); + assert!(current_timestamp + 2 <= marked_splits[1].update_timestamp); + + assert_eq!(marked_splits[2].split_id(), split_id_3); + assert_eq!(marked_splits[2].update_timestamp, split_3_update_timestamp); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_delete_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-delete-splits"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let delete_splits_request = DeleteSplitsRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + split_ids: Vec::new(), + }; + let error = metastore + .delete_splits(delete_splits_request) + .await + .unwrap_err(); + + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let index_not_existing_uid = IndexUid::new_with_random_ulid(&index_id); + // Check error if index does not exist. + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_not_existing_uid.to_string(), + split_ids: Vec::new(), + }; + let error = metastore + .delete_splits(delete_splits_request) + .await + .unwrap_err(); + + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.clone().into(), + split_ids: vec!["split-not-found".to_string()], + }; + metastore + .delete_splits(delete_splits_request) + .await + .unwrap(); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_1.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + ..Default::default() + }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata_2.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.clone().into(), + split_ids: vec![split_id_1.clone(), split_id_2.clone()], + }; + let error = metastore + .delete_splits(delete_splits_request) + .await + .unwrap_err(); + + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + )); + + assert_eq!( + metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap() + .len(), + 2 + ); + + let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( + index_uid.clone(), + vec![split_id_1.clone(), split_id_2.clone()], + ); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.clone().into(), + split_ids: vec![ + split_id_1.clone(), + split_id_2.clone(), + "split-not-found".to_string(), + ], + }; + metastore + .delete_splits(delete_splits_request) + .await + .unwrap(); + + assert_eq!( + metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap() + .len(), + 0 + ); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_split_update_timestamp< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let mut current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + + let index_id = append_random_suffix("split-update-timestamp"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let source_id = format!("{index_id}--source"); + + let split_id = format!("{index_id}--split"); + let split_metadata = SplitMetadata { + split_id: split_id.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + ..Default::default() + }; + + // Create an index + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + // wait for 1s, stage split & check `update_timestamp` + sleep(Duration::from_secs(1)).await; + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + sleep(Duration::from_secs(1)).await; + let split_meta = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap()[0] + .clone(); + assert!(split_meta.update_timestamp > current_timestamp); + assert!(split_meta.publish_timestamp.is_none()); + + current_timestamp = split_meta.update_timestamp; + + // wait for 1s, publish split & check `update_timestamp` + sleep(Duration::from_secs(1)).await; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id.clone()], + index_checkpoint_delta_json_opt: Some({ + let offsets = 0..5; + let checkpoint_delta = IndexCheckpointDelta::for_test(&source_id, offsets); + serde_json::to_string(&checkpoint_delta).unwrap() + }), + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + let split_meta = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap()[0] + .clone(); + assert!(split_meta.update_timestamp > current_timestamp); + assert_eq!( + split_meta.publish_timestamp, + Some(split_meta.update_timestamp) + ); + current_timestamp = split_meta.update_timestamp; + + // wait for 1s, mark split for deletion & check `update_timestamp` + sleep(Duration::from_secs(1)).await; + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + let split_meta = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap()[0] + .clone(); + assert!(split_meta.update_timestamp > current_timestamp); + assert!(split_meta.publish_timestamp.is_some()); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_stage_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + let index_id = append_random_suffix("test-stage-splits"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 20, + ..Default::default() + }; + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 10, + ..Default::default() + }; + + // Stage a splits on a non-existent index + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + IndexUid::new_with_random_ulid("index-not-found"), + vec![split_metadata_1.clone()], + ) + .unwrap(); + let error = metastore + .stage_splits(stage_splits_request) + .await + .unwrap_err(); + assert!(matches!( + error, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + // Stage a split on an index + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone(), split_metadata_2.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + let split_ids = collect_split_ids(&splits); + assert_eq!(split_ids, &[&split_id_1, &split_id_2]); + + // Stage a existent-staged-split on an index + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone()], + ) + .unwrap(); + metastore + .stage_splits(stage_splits_request) + .await + .expect("Pre-existing staged splits should be updated."); + + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone()], + ) + .unwrap(); + let error = metastore + .stage_splits(stage_splits_request) + .await + .expect_err("Metastore should not allow splits which are not `Staged` to be overwritten."); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + ),); + + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.clone(), vec![split_id_2.clone()]); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await + .unwrap(); + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_2.clone()], + ) + .unwrap(); + let error = metastore + .stage_splits(stage_splits_request) + .await + .expect_err("Metastore should not allow splits which are not `Staged` to be overwritten."); + assert!(matches!( + error, + MetastoreError::FailedPrecondition { + entity: EntityKind::Splits { .. }, + .. + } + ),); + + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_update_splits_delete_opstamp< + MetastoreToTest: MetastoreServiceExt + DefaultForTest, +>() { + let mut metastore = MetastoreToTest::default_for_test().await; + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + let index_id = append_random_suffix("update-splits-delete-opstamp"); + let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 20, + ..Default::default() + }; + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 10, + ..Default::default() + }; + let split_id_3 = format!("{index_id}--split-3"); + let split_metadata_3 = SplitMetadata { + split_id: split_id_3.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 0, + ..Default::default() + }; + + { + info!("update splits delete opstamp on a non-existent index"); + let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { + index_uid: IndexUid::new_with_random_ulid("index-not-found").to_string(), + split_ids: vec![split_id_1.clone()], + delete_opstamp: 10, + }; + let metastore_err = metastore + .update_splits_delete_opstamp(update_splits_delete_opstamp_request) + .await + .unwrap_err(); + error!(err=?metastore_err); + assert!(matches!( + metastore_err, + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + } + + { + info!("update splits delete opstamp on an index"); + let create_index_request = + CreateIndexRequest::try_from_index_config(index_config.clone()).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .into(); + + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![ + split_metadata_1.clone(), + split_metadata_2.clone(), + split_metadata_3.clone(), + ], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.clone().into(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; + metastore + .publish_splits(publish_splits_request) + .await + .unwrap(); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 100, + num_splits: 2, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 2); + + let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { + index_uid: index_uid.clone().into(), + split_ids: vec![split_id_1.clone(), split_id_2.clone()], + delete_opstamp: 100, + }; + metastore + .update_splits_delete_opstamp(update_splits_delete_opstamp_request) + .await + .unwrap(); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 100, + num_splits: 2, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 0); + + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.clone().into(), + delete_opstamp: 200, + num_splits: 2, + }; + let splits = metastore + .list_stale_splits(list_stale_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); + assert_eq!(splits.len(), 2); + assert_eq!(splits[0].split_metadata.delete_opstamp, 100); + assert_eq!(splits[1].split_metadata.delete_opstamp, 100); + + cleanup_index(&mut metastore, index_uid).await; + } +} diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 431414bae1..970d605b38 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -100,8 +100,8 @@ service MetastoreService { // Deletes an index rpc DeleteIndex(DeleteIndexRequest) returns (EmptyResponse); - // Gets splits from index. - rpc ListSplits(ListSplitsRequest) returns (ListSplitsResponse); + // Streams splits from index. + rpc ListSplits(ListSplitsRequest) returns (stream ListSplitsResponse); // Stages several splits. rpc StageSplits(StageSplitsRequest) returns (EmptyResponse); diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index c1a0ace27f..84eb8f9c24 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -572,6 +572,9 @@ impl PrometheusLabels<1> for ListShardsRequest { OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_shards")]) } } +pub type MetastoreServiceStream = quickwit_common::ServiceStream< + crate::metastore::MetastoreResult, +>; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { @@ -598,11 +601,11 @@ pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync &mut self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult; - /// Gets splits from index. + /// Streams splits from index. async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> crate::metastore::MetastoreResult; + ) -> crate::metastore::MetastoreResult>; /// Stages several splits. async fn stage_splits( &mut self, @@ -804,7 +807,7 @@ impl MetastoreService for MetastoreServiceClient { async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> crate::metastore::MetastoreResult { + ) -> crate::metastore::MetastoreResult> { self.inner.list_splits(request).await } async fn stage_splits( @@ -952,7 +955,9 @@ pub mod metastore_service_mock { async fn list_splits( &mut self, request: super::ListSplitsRequest, - ) -> crate::metastore::MetastoreResult { + ) -> crate::metastore::MetastoreResult< + MetastoreServiceStream, + > { self.inner.lock().await.list_splits(request).await } async fn stage_splits( @@ -1143,7 +1148,7 @@ impl tower::Service for Box { } } impl tower::Service for Box { - type Response = ListSplitsResponse; + type Response = MetastoreServiceStream; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; fn poll_ready( @@ -1456,7 +1461,7 @@ struct MetastoreServiceTowerBlock { >, list_splits_svc: quickwit_common::tower::BoxService< ListSplitsRequest, - ListSplitsResponse, + MetastoreServiceStream, crate::metastore::MetastoreError, >, stage_splits_svc: quickwit_common::tower::BoxService< @@ -1605,7 +1610,7 @@ impl MetastoreService for MetastoreServiceTowerBlock { async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> crate::metastore::MetastoreResult { + ) -> crate::metastore::MetastoreResult> { self.list_splits_svc.ready().await?.call(request).await } async fn stage_splits( @@ -1760,7 +1765,7 @@ pub struct MetastoreServiceTowerBlockBuilder { quickwit_common::tower::BoxLayer< Box, ListSplitsRequest, - ListSplitsResponse, + MetastoreServiceStream, crate::metastore::MetastoreError, >, >, @@ -1950,7 +1955,7 @@ impl MetastoreServiceTowerBlockBuilder { >::Future: Send + 'static, L::Service: tower::Service< ListSplitsRequest, - Response = ListSplitsResponse, + Response = MetastoreServiceStream, Error = crate::metastore::MetastoreError, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, @@ -2212,7 +2217,7 @@ impl MetastoreServiceTowerBlockBuilder { L: tower::Layer> + Send + Sync + 'static, L::Service: tower::Service< ListSplitsRequest, - Response = ListSplitsResponse, + Response = MetastoreServiceStream, Error = crate::metastore::MetastoreError, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, @@ -2753,9 +2758,12 @@ where > + tower::Service< ListSplitsRequest, - Response = ListSplitsResponse, + Response = MetastoreServiceStream, Error = crate::metastore::MetastoreError, - Future = BoxFuture, + Future = BoxFuture< + MetastoreServiceStream, + crate::metastore::MetastoreError, + >, > + tower::Service< StageSplitsRequest, @@ -2893,7 +2901,7 @@ where async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> crate::metastore::MetastoreResult { + ) -> crate::metastore::MetastoreResult> { self.call(request).await } async fn stage_splits( @@ -3088,11 +3096,15 @@ where async fn list_splits( &mut self, request: ListSplitsRequest, - ) -> crate::metastore::MetastoreResult { + ) -> crate::metastore::MetastoreResult> { self.inner .list_splits(request) .await - .map(|response| response.into_inner()) + .map(|response| { + let streaming: tonic::Streaming<_> = response.into_inner(); + let stream = quickwit_common::ServiceStream::from(streaming); + stream.map_err(|error| error.into()) + }) .map_err(|error| error.into()) } async fn stage_splits( @@ -3340,15 +3352,18 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } + type ListSplitsStream = quickwit_common::ServiceStream< + tonic::Result, + >; async fn list_splits( &self, request: tonic::Request, - ) -> Result, tonic::Status> { + ) -> Result, tonic::Status> { self.inner .clone() .list_splits(request.into_inner()) .await - .map(tonic::Response::new) + .map(|stream| tonic::Response::new(stream.map_err(|error| error.into()))) .map_err(|error| error.into()) } async fn stage_splits( @@ -3788,12 +3803,12 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } - /// Gets splits from index. + /// Streams splits from index. pub async fn list_splits( &mut self, request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response>, tonic::Status, > { self.inner @@ -3814,7 +3829,7 @@ pub mod metastore_service_grpc_client { .insert( GrpcMethod::new("quickwit.metastore.MetastoreService", "ListSplits"), ); - self.inner.unary(req, path, codec).await + self.inner.server_streaming(req, path, codec).await } /// Stages several splits. pub async fn stage_splits( @@ -4352,14 +4367,17 @@ pub mod metastore_service_grpc_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; - /// Gets splits from index. + /// Server streaming response type for the ListSplits method. + type ListSplitsStream: futures_core::Stream< + Item = std::result::Result, + > + + Send + + 'static; + /// Streams splits from index. async fn list_splits( &self, request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; + ) -> std::result::Result, tonic::Status>; /// Stages several splits. async fn stage_splits( &self, @@ -4792,11 +4810,12 @@ pub mod metastore_service_grpc_server { struct ListSplitsSvc(pub Arc); impl< T: MetastoreServiceGrpc, - > tonic::server::UnaryService + > tonic::server::ServerStreamingService for ListSplitsSvc { type Response = super::ListSplitsResponse; + type ResponseStream = T::ListSplitsStream; type Future = BoxFuture< - tonic::Response, + tonic::Response, tonic::Status, >; fn call( @@ -4826,7 +4845,7 @@ pub mod metastore_service_grpc_server { max_decoding_message_size, max_encoding_message_size, ); - let res = grpc.unary(method, req).await; + let res = grpc.server_streaming(method, req).await; Ok(res) }; Box::pin(fut) diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 6162b37d75..65fc51ff29 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -62,7 +62,8 @@ pub use find_trace_ids_collector::FindTraceIdsCollector; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::tag_pruning::TagFilterAst; use quickwit_metastore::{ - ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitMetadata, + SplitState, }; use quickwit_proto::search::{PartialHit, SearchRequest, SearchResponse, SplitIdAndFooterOffsets}; use quickwit_proto::types::IndexUid; @@ -189,10 +190,8 @@ async fn list_relevant_splits( let splits_metadata: Vec = metastore .list_splits(list_splits_request) .await? - .deserialize_splits()? - .into_iter() - .map(|split| split.split_metadata) - .collect(); + .collect_splits_metadata() + .await?; Ok(splits_metadata) } diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 07489720df..05252a6596 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -31,7 +31,7 @@ use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; use quickwit_doc_mapper::{DocMapper, DYNAMIC_FIELD_NAME}; use quickwit_metastore::{ IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsRequestExt, - ListSplitsResponseExt, SplitMetadata, + MetastoreServiceStreamSplitsExt, SplitMetadata, }; use quickwit_proto::metastore::{ IndexMetadataRequest, ListIndexesMetadataRequest, ListSplitsRequest, MetastoreService, @@ -1016,7 +1016,8 @@ pub async fn root_list_terms( .clone() .list_splits(list_splits_request) .await? - .deserialize_splits_metadata()?; + .collect_splits_metadata() + .await?; let index_uri = &index_config.index_uri; @@ -1212,9 +1213,10 @@ mod tests { use std::sync::{Arc, RwLock}; use quickwit_common::shared_consts::SCROLL_BATCH_LEN; + use quickwit_common::ServiceStream; use quickwit_config::{DocMapping, IndexingSettings, SearchSettings}; use quickwit_indexing::MockSplitBuilder; - use quickwit_metastore::IndexMetadata; + use quickwit_metastore::{IndexMetadata, ListSplitsResponseExt}; use quickwit_proto::metastore::{ListIndexesMetadataResponse, ListSplitsResponse}; use quickwit_proto::search::{ScrollRequest, SortOrder, SortValue, SplitSearchError}; use quickwit_query::query_ast::{qast_helper, qast_json_helper, query_ast_from_user_text}; @@ -1474,7 +1476,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( @@ -1566,7 +1569,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().returning( @@ -1637,7 +1641,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -1732,7 +1737,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -1911,7 +1917,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -2085,7 +2092,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); @@ -2205,7 +2213,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1 @@ -2332,7 +2341,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut first_call = true; let mut mock_search_service = MockSearchService::new(); @@ -2409,7 +2419,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service = MockSearchService::new(); @@ -2471,7 +2482,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); // Service1 - broken node. let mut mock_search_service_1 = MockSearchService::new(); @@ -2559,7 +2571,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); // Service1 - working node. @@ -2632,7 +2645,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); @@ -2715,7 +2729,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); @@ -2763,7 +2778,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); @@ -2967,7 +2983,8 @@ mod tests { .with_index_uid(&index_uid_2) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().times(2).returning( @@ -3172,7 +3189,8 @@ mod tests { .with_index_uid(&index_uid_2) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits_response)])) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1 diff --git a/quickwit/quickwit-search/src/search_stream/leaf.rs b/quickwit/quickwit-search/src/search_stream/leaf.rs index 40bce90f37..60b32b861c 100644 --- a/quickwit/quickwit-search/src/search_stream/leaf.rs +++ b/quickwit/quickwit-search/src/search_stream/leaf.rs @@ -451,7 +451,7 @@ mod tests { use itertools::Itertools; use quickwit_indexing::TestSandbox; - use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; + use quickwit_metastore::{ListSplitsRequestExt, MetastoreServiceStreamSplitsExt}; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; use quickwit_query::query_ast::qast_json_helper; use serde_json::json; @@ -502,7 +502,9 @@ mod tests { .metastore() .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await? - .deserialize_splits()?; + .collect_splits() + .await + .unwrap(); let splits_offsets = splits .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) @@ -579,7 +581,8 @@ mod tests { .metastore() .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await? - .deserialize_splits()?; + .collect_splits() + .await?; let splits_offsets = splits .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) @@ -635,7 +638,8 @@ mod tests { .metastore() .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await? - .deserialize_splits()?; + .collect_splits() + .await?; let splits_offsets = splits .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) @@ -724,7 +728,8 @@ mod tests { .metastore() .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await? - .deserialize_splits()?; + .collect_splits() + .await?; let splits_offsets = splits .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) diff --git a/quickwit/quickwit-search/src/search_stream/root.rs b/quickwit/quickwit-search/src/search_stream/root.rs index 41daf54840..2ba006720e 100644 --- a/quickwit/quickwit-search/src/search_stream/root.rs +++ b/quickwit/quickwit-search/src/search_stream/root.rs @@ -132,6 +132,7 @@ fn jobs_to_leaf_request( #[cfg(test)] mod tests { + use quickwit_common::ServiceStream; use quickwit_indexing::MockSplitBuilder; use quickwit_metastore::{IndexMetadata, ListSplitsResponseExt}; use quickwit_proto::metastore::{IndexMetadataResponse, ListSplitsResponse}; @@ -161,7 +162,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -218,7 +220,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -278,7 +281,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -333,7 +337,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split") .with_index_uid(&index_uid) .build()]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); diff --git a/quickwit/quickwit-search/src/tests.rs b/quickwit/quickwit-search/src/tests.rs index 6f84651a62..d84b730f1e 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -1029,7 +1029,8 @@ async fn test_search_util(test_sandbox: &TestSandbox, query: &str) -> Vec { .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await .unwrap() - .deserialize_splits() + .collect_splits() + .await .unwrap(); let splits_offsets: Vec<_> = splits .into_iter() @@ -1667,7 +1668,9 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { .metastore() .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await? - .deserialize_splits()?; + .collect_splits() + .await + .unwrap(); let splits_offsets: Vec<_> = splits .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index 78d235fe61..4a8b6fc71b 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -30,7 +30,7 @@ use quickwit_doc_mapper::{analyze_text, TokenizerConfig}; use quickwit_index_management::{IndexService, IndexServiceError}; use quickwit_metastore::{ IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, - ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitInfo, SplitState, + ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitInfo, SplitState, }; use quickwit_proto::metastore::{ DeleteSourceRequest, EntityKind, IndexMetadataRequest, ListIndexesMetadataRequest, @@ -201,7 +201,8 @@ async fn describe_index( let splits = metastore .list_splits(list_splits_request) .await? - .deserialize_splits()?; + .collect_splits() + .await?; let published_splits: Vec = splits .into_iter() .filter(|split| split.split_state == SplitState::Published) @@ -350,7 +351,8 @@ async fn list_splits( let splits = metastore .list_splits(list_splits_request) .await? - .deserialize_splits()?; + .collect_splits() + .await?; Ok(ListSplitsResponse { offset, size: splits.len(), @@ -869,9 +871,10 @@ mod tests { use assert_json_diff::assert_json_include; use quickwit_common::uri::Uri; + use quickwit_common::ServiceStream; use quickwit_config::{SourceParams, VecSourceParams}; use quickwit_indexing::{mock_split, MockSplitBuilder}; - use quickwit_metastore::{metastore_for_test, IndexMetadata}; + use quickwit_metastore::{metastore_for_test, IndexMetadata, ListSplitsResponseExt}; use quickwit_proto::metastore::{ EmptyResponse, IndexMetadataResponse, ListIndexesMetadataResponse, ListSplitsResponse, MetastoreServiceClient, SourceType, @@ -958,7 +961,8 @@ mod tests { let splits = vec![MockSplitBuilder::new("split_1") .with_index_uid(&index_uid) .build()]; - return Ok(ListSplitsResponse::try_from_splits(splits).unwrap()); + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + return Ok(ServiceStream::from(vec![Ok(splits)])); } Err(MetastoreError::Internal { message: "".to_string(), @@ -1038,7 +1042,8 @@ mod tests { }) .return_once(move |_| { let splits = vec![split_1, split_2]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let index_service = IndexService::new( @@ -1090,7 +1095,7 @@ mod tests { && list_split_query.time_range.is_unbounded() && list_split_query.create_timestamp.is_unbounded() { - return Ok(ListSplitsResponse::empty()); + return Ok(ServiceStream::empty()); } Err(MetastoreError::Internal { message: "".to_string(), @@ -1224,7 +1229,8 @@ mod tests { ) }); mock_metastore.expect_list_splits().return_once(|_| { - Ok(ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap()) + let splits = ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); mock_metastore .expect_mark_splits_for_deletion() @@ -1269,7 +1275,9 @@ mod tests { mock_metastore .expect_list_splits() .returning(|_| { - Ok(ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap()) + let splits = + ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }) .times(3); mock_metastore diff --git a/quickwit/quickwit-serve/src/search_api/mod.rs b/quickwit/quickwit-serve/src/search_api/mod.rs index b77cc5d919..0bc69b4406 100644 --- a/quickwit/quickwit-serve/src/search_api/mod.rs +++ b/quickwit/quickwit-serve/src/search_api/mod.rs @@ -33,6 +33,7 @@ mod tests { use std::sync::Arc; use futures::TryStreamExt; + use quickwit_common::ServiceStream; use quickwit_indexing::MockSplitBuilder; use quickwit_metastore::{IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt}; use quickwit_proto::metastore::{ @@ -94,7 +95,8 @@ mod tests { .with_index_uid(&index_uid) .build(), ]; - Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + let splits = ListSplitsResponse::try_from_splits(splits).unwrap(); + Ok(ServiceStream::from(vec![Ok(splits)])) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel();