From 4d83afd74666114fe660d232f26ceb2f6e7bc632 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Massot?= Date: Sat, 21 Oct 2023 21:32:00 +0200 Subject: [PATCH] Migrate metastore to code generated metastore service (#3898) * Migrate the metastore trait to the code generated metastore service and client. * Add changes from review's suggestions. * Last clean before merge. * Readd metastore storage connectivity check. --- quickwit/Cargo.lock | 3 + quickwit/quickwit-cli/src/lib.rs | 29 +- quickwit/quickwit-cli/src/tool.rs | 26 +- quickwit/quickwit-cli/tests/cli.rs | 122 +- quickwit/quickwit-cli/tests/helpers.rs | 12 +- quickwit/quickwit-codegen/Cargo.toml | 1 + quickwit/quickwit-codegen/example/Cargo.toml | 1 + quickwit/quickwit-codegen/example/build.rs | 1 + .../example/src/codegen/hello.rs | 18 +- quickwit/quickwit-codegen/src/codegen.rs | 68 +- .../src/tower/event_listener.rs | 1 + quickwit/quickwit-common/src/tower/mod.rs | 1 + quickwit/quickwit-common/src/tower/retry.rs | 31 +- .../src/control_plane.rs | 265 +- .../src/control_plane_model.rs | 33 +- .../src/indexing_scheduler.rs | 9 +- .../src/ingest/ingest_controller.rs | 52 +- quickwit/quickwit-control-plane/src/tests.rs | 18 +- quickwit/quickwit-index-management/Cargo.toml | 1 + .../src/garbage_collection.rs | 293 +- .../quickwit-index-management/src/index.rs | 197 +- quickwit/quickwit-indexing/failpoints/mod.rs | 28 +- .../quickwit-indexing/src/actors/indexer.rs | 140 +- .../src/actors/indexing_pipeline.rs | 217 +- .../src/actors/indexing_service.rs | 177 +- .../src/actors/merge_executor.rs | 100 +- .../src/actors/merge_pipeline.rs | 40 +- .../quickwit-indexing/src/actors/publisher.rs | 136 +- .../quickwit-indexing/src/actors/uploader.rs | 65 +- quickwit/quickwit-indexing/src/lib.rs | 6 +- .../src/source/ingest/mod.rs | 24 +- .../src/source/kafka_source.rs | 57 +- quickwit/quickwit-indexing/src/source/mod.rs | 10 +- .../src/source/pulsar_source.rs | 45 +- quickwit/quickwit-indexing/src/test_utils.rs | 37 +- quickwit/quickwit-ingest/build.rs | 1 + .../src/codegen/ingest_service.rs | 6 + .../quickwit-jaeger/src/integration_tests.rs | 31 +- quickwit/quickwit-janitor/Cargo.toml | 1 + .../src/actors/delete_task_pipeline.rs | 31 +- .../src/actors/delete_task_planner.rs | 101 +- .../src/actors/delete_task_service.rs | 38 +- .../src/actors/garbage_collector.rs | 262 +- .../src/actors/retention_policy_executor.rs | 138 +- quickwit/quickwit-janitor/src/lib.rs | 7 +- .../src/retention_policy_execution.rs | 25 +- .../quickwit-metastore-utils/src/bin/proxy.rs | 26 +- .../src/bin/replay.rs | 7 +- .../src/grpc_request.rs | 3 +- quickwit/quickwit-metastore/src/lib.rs | 33 +- .../src/metastore/control_plane_metastore.rs | 256 +- .../file_backed_index/mod.rs | 101 +- .../file_backed_metastore_factory.rs | 49 +- .../metastore/file_backed_metastore/mod.rs | 792 ++-- .../metastore/grpc_metastore/grpc_adapter.rs | 517 --- .../src/metastore/grpc_metastore/mod.rs | 670 --- .../src/metastore/instrumented_metastore.rs | 390 -- .../metastore/metastore_event_publisher.rs | 349 -- .../quickwit-metastore/src/metastore/mod.rs | 661 +-- .../src/metastore/postgresql_metastore.rs | 367 +- .../src/metastore/retrying_metastore/mod.rs | 331 -- .../src/metastore/retrying_metastore/retry.rs | 141 - .../src/metastore/retrying_metastore/test.rs | 374 -- .../src/metastore_factory.rs | 15 +- .../src/metastore_resolver.rs | 14 +- quickwit/quickwit-metastore/src/metrics.rs | 56 - quickwit/quickwit-metastore/src/tests.rs | 2590 +++++++---- .../quickwit-opentelemetry/src/otlp/logs.rs | 8 +- .../quickwit-opentelemetry/src/otlp/traces.rs | 8 +- quickwit/quickwit-proto/Cargo.toml | 4 +- quickwit/quickwit-proto/build.rs | 90 +- .../protos/quickwit/metastore.proto | 52 +- .../quickwit/quickwit.control_plane.rs | 6 + .../src/codegen/quickwit/quickwit.indexing.rs | 6 + .../quickwit/quickwit.ingest.ingester.rs | 6 + .../quickwit/quickwit.ingest.router.rs | 6 + .../src/codegen/quickwit/quickwit.ingest.rs | 10 +- .../codegen/quickwit/quickwit.metastore.rs | 3901 +++++++++++++++-- .../quickwit-proto/src/metastore/events.rs | 11 +- quickwit/quickwit-proto/src/metastore/mod.rs | 74 +- quickwit/quickwit-search/Cargo.toml | 1 + quickwit/quickwit-search/src/lib.rs | 23 +- quickwit/quickwit-search/src/root.rs | 408 +- .../quickwit-search/src/search_stream/leaf.rs | 30 +- .../quickwit-search/src/search_stream/root.rs | 110 +- quickwit/quickwit-search/src/service.rs | 12 +- quickwit/quickwit-search/src/tests.rs | 20 +- quickwit/quickwit-serve/Cargo.toml | 1 + .../src/delete_task_api/handler.rs | 38 +- .../src/elastic_search_api/bulk.rs | 2 +- quickwit/quickwit-serve/src/grpc.rs | 5 +- .../src/index_api/rest_handler.rs | 586 ++- .../quickwit-serve/src/ingest_metastore.rs | 10 +- quickwit/quickwit-serve/src/lib.rs | 174 +- quickwit/quickwit-serve/src/search_api/mod.rs | 27 +- 95 files changed, 9526 insertions(+), 6750 deletions(-) delete mode 100644 quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/retrying_metastore/retry.rs delete mode 100644 quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs delete mode 100644 quickwit/quickwit-metastore/src/metrics.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 2a27ca749d2..2209bd19854 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -5025,6 +5025,7 @@ name = "quickwit-codegen" version = "0.6.3" dependencies = [ "anyhow", + "futures", "heck", "prettyplease 0.2.15", "proc-macro2", @@ -5051,6 +5052,7 @@ dependencies = [ "quickwit-actors", "quickwit-codegen", "quickwit-common", + "quickwit-macros", "serde", "thiserror", "tokio", @@ -5558,6 +5560,7 @@ dependencies = [ "async-trait", "bytes", "dyn-clone", + "futures", "glob", "http", "hyper", diff --git a/quickwit/quickwit-cli/src/lib.rs b/quickwit/quickwit-cli/src/lib.rs index 1d90143c13b..3c5cdafc366 100644 --- a/quickwit/quickwit-cli/src/lib.rs +++ b/quickwit/quickwit-cli/src/lib.rs @@ -34,7 +34,8 @@ use quickwit_config::{ DEFAULT_QW_CONFIG_PATH, }; use quickwit_indexing::check_source_connectivity; -use quickwit_metastore::{Metastore, MetastoreResolver}; +use quickwit_metastore::{IndexMetadataResponseExt, MetastoreResolver}; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_rest_client::models::Timeout; use quickwit_rest_client::rest_client::{QuickwitClient, QuickwitClientBuilder, DEFAULT_BASE_URL}; use quickwit_storage::{load_file, StorageResolver}; @@ -256,23 +257,29 @@ fn get_resolvers( /// Optionally, it takes a `SourceConfig` that will be checked instead /// of the index's sources. pub async fn run_index_checklist( - metastore: &dyn Metastore, + metastore: &mut MetastoreServiceClient, storage_resolver: &StorageResolver, index_id: &str, source_config_opt: Option<&SourceConfig>, ) -> anyhow::Result<()> { let mut checks: Vec<(&str, anyhow::Result<()>)> = Vec::new(); - - // The metastore is file-backed, so we must check the storage first. - if !metastore.uri().protocol().is_database() { - let metastore_storage = storage_resolver.resolve(metastore.uri()).await?; - checks.push(( - "metastore storage", - metastore_storage.check_connectivity().await, - )); + for metastore_endpoint in metastore.endpoints() { + // If it's not a database, the metastore is file-backed. To display a nicer message to the + // user, we check the metastore storage connectivity before the mestastore check + // connectivity which will check the storage anyway. + if !metastore_endpoint.protocol().is_database() { + let metastore_storage = storage_resolver.resolve(&metastore_endpoint).await?; + checks.push(( + "metastore storage", + metastore_storage.check_connectivity().await, + )); + } } checks.push(("metastore", metastore.check_connectivity().await)); - let index_metadata = metastore.index_metadata(index_id).await?; + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await? + .deserialize_index_metadata()?; let index_storage = storage_resolver.resolve(index_metadata.index_uri()).await?; checks.push(("index storage", index_storage.check_connectivity().await)); diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 1559ea4abe6..76aa223df5f 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -22,7 +22,6 @@ use std::io::{stdout, IsTerminal, Stdout, Write}; use std::num::NonZeroUsize; use std::path::PathBuf; use std::str::FromStr; -use std::sync::Arc; use std::time::{Duration, Instant}; use std::{env, fmt, io}; @@ -49,7 +48,8 @@ use quickwit_indexing::models::{ }; use quickwit_indexing::IndexingPipeline; use quickwit_ingest::IngesterPool; -use quickwit_metastore::Metastore; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::search::SearchResponse; use quickwit_proto::NodeId; use quickwit_search::{single_node_search, SearchResponseRest}; @@ -409,7 +409,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result< let config = load_node_config(&args.config_uri).await?; let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); - let metastore = metastore_resolver.resolve(&config.metastore_uri).await?; + let mut metastore = metastore_resolver.resolve(&config.metastore_uri).await?; let source_params = if let Some(filepath) = args.input_path_opt.as_ref() { SourceParams::file(filepath) @@ -429,7 +429,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result< input_format: args.input_format, }; run_index_checklist( - &*metastore, + &mut metastore, &storage_resolver, &args.index_id, Some(&source_config), @@ -437,7 +437,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result< .await?; if args.overwrite { - let index_service = IndexService::new(metastore.clone(), storage_resolver.clone()); + let mut index_service = IndexService::new(metastore.clone(), storage_resolver.clone()); index_service.clear_index(&args.index_id).await?; } // The indexing service needs to update its cluster chitchat state so that the control plane is @@ -533,7 +533,8 @@ pub async fn local_search_cli(args: LocalSearchArgs) -> anyhow::Result<()> { let config = load_node_config(&args.config_uri).await?; let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); - let metastore: Arc = metastore_resolver.resolve(&config.metastore_uri).await?; + let metastore: MetastoreServiceClient = + metastore_resolver.resolve(&config.metastore_uri).await?; let aggs = args .aggregation .map(|agg_string| serde_json::from_str(&agg_string)) @@ -568,8 +569,8 @@ pub async fn merge_cli(args: MergeArgs) -> anyhow::Result<()> { let config = load_node_config(&args.config_uri).await?; let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); - let metastore = metastore_resolver.resolve(&config.metastore_uri).await?; - run_index_checklist(&*metastore, &storage_resolver, &args.index_id, None).await?; + let mut metastore = metastore_resolver.resolve(&config.metastore_uri).await?; + run_index_checklist(&mut metastore, &storage_resolver, &args.index_id, None).await?; // The indexing service needs to update its cluster chitchat state so that the control plane is // aware of the running tasks. We thus create a fake cluster to instantiate the indexing service // and avoid impacting potential control plane running on the cluster. @@ -657,7 +658,7 @@ pub async fn garbage_collect_index_cli(args: GarbageCollectIndexArgs) -> anyhow: let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); let metastore = metastore_resolver.resolve(&config.metastore_uri).await?; - let index_service = IndexService::new(metastore, storage_resolver); + let mut index_service = IndexService::new(metastore, storage_resolver); let removal_info = index_service .garbage_collect_index(&args.index_id, args.grace_period, args.dry_run) .await?; @@ -721,8 +722,11 @@ async fn extract_split_cli(args: ExtractSplitArgs) -> anyhow::Result<()> { let config = load_node_config(&args.config_uri).await?; let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); - let metastore = metastore_resolver.resolve(&config.metastore_uri).await?; - let index_metadata = metastore.index_metadata(&args.index_id).await?; + let mut metastore = metastore_resolver.resolve(&config.metastore_uri).await?; + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(args.index_id)) + .await? + .deserialize_index_metadata()?; let index_storage = storage_resolver.resolve(index_metadata.index_uri()).await?; let split_file = PathBuf::from(format!("{}.split", args.split_id)); let split_data = index_storage.get_all(split_file.as_path()).await?; diff --git a/quickwit/quickwit-cli/tests/cli.rs b/quickwit/quickwit-cli/tests/cli.rs index 0250caff9fb..9460a72a942 100644 --- a/quickwit/quickwit-cli/tests/cli.rs +++ b/quickwit/quickwit-cli/tests/cli.rs @@ -41,8 +41,14 @@ use quickwit_common::fs::get_cache_directory_path; use quickwit_common::rand::append_random_suffix; use quickwit_common::uri::Uri; use quickwit_config::{SourceInputFormat, CLI_INGEST_SOURCE_ID}; -use quickwit_metastore::{MetastoreResolver, SplitState}; -use quickwit_proto::metastore::{EntityKind, MetastoreError}; +use quickwit_metastore::{ + ListSplitsRequestExt, ListSplitsResponseExt, MetastoreResolver, MetastoreServiceExt, + SplitState, StageSplitsRequestExt, +}; +use quickwit_proto::metastore::{ + DeleteSplitsRequest, EntityKind, IndexMetadataRequest, ListSplitsRequest, + MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, StageSplitsRequest, +}; use serde_json::{json, Number, Value}; use tokio::time::{sleep, Duration}; @@ -246,8 +252,10 @@ async fn test_ingest_docs_cli() { let splits: Vec<_> = test_env .metastore() .await - .list_all_splits(index_uid) + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await + .unwrap() + .deserialize_splits() .unwrap(); assert_eq!(splits.len(), 1); @@ -556,7 +564,7 @@ async fn test_delete_index_cli_dry_run() { assume_yes: true, }; - let metastore = MetastoreResolver::unconfigured() + let mut metastore = MetastoreResolver::unconfigured() .resolve(&test_env.metastore_uri) .await .unwrap(); @@ -567,7 +575,11 @@ async fn test_delete_index_cli_dry_run() { delete_index_cli(args).await.unwrap(); // On dry run index should still exist - let metastore = refresh_metastore(metastore).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap(); assert!(metastore.index_exists(&index_id).await.unwrap()); local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) @@ -579,7 +591,11 @@ async fn test_delete_index_cli_dry_run() { delete_index_cli(args).await.unwrap(); // On dry run index should still exist - let metastore = refresh_metastore(metastore).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap(); assert!(metastore.index_exists(&index_id).await.unwrap()); } @@ -625,7 +641,7 @@ async fn test_garbage_collect_cli_no_grace() { .await .unwrap(); - let metastore = MetastoreResolver::unconfigured() + let mut metastore = MetastoreResolver::unconfigured() .resolve(&test_env.metastore_uri) .await .unwrap(); @@ -649,7 +665,12 @@ async fn test_garbage_collect_cli_no_grace() { dry_run, }; - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 1); let args = create_gc_args(false); @@ -660,10 +681,12 @@ 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 = [splits[0].split_id()]; - let metastore = refresh_metastore(metastore).await.unwrap(); + let split_ids = vec![splits[0].split_id().to_string()]; + let mut metastore = refresh_metastore(metastore).await.unwrap(); + let mark_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), split_ids.clone()); metastore - .mark_splits_for_deletion(index_uid.clone(), &split_ids) + .mark_splits_for_deletion(mark_for_deletion_request) .await .unwrap(); @@ -672,7 +695,7 @@ async fn test_garbage_collect_cli_no_grace() { garbage_collect_index_cli(args).await.unwrap(); // On `dry_run = true` splits `MarkedForDeletion` should still exist. - for split_id in split_ids { + for split_id in split_ids.iter() { let split_file = quickwit_common::split_file(split_id); let split_filepath = index_path.join(split_file); assert_eq!(split_filepath.try_exists().unwrap(), true); @@ -683,18 +706,20 @@ async fn test_garbage_collect_cli_no_grace() { garbage_collect_index_cli(args).await.unwrap(); // If split is `MarkedForDeletion` it should be deleted after gc run - for split_id in split_ids { + for split_id in split_ids.iter() { let split_file = quickwit_common::split_file(split_id); let split_filepath = index_path.join(split_file); assert_eq!(split_filepath.try_exists().unwrap(), false); } - let metastore = refresh_metastore(metastore).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); assert_eq!( metastore - .list_all_splits(index_uid.clone()) + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() + .deserialize_splits() + .unwrap() .len(), 0 ); @@ -746,12 +771,17 @@ async fn test_garbage_collect_index_cli() { dry_run: false, }; - let metastore = MetastoreResolver::unconfigured() + let mut metastore = MetastoreResolver::unconfigured() .resolve(&test_env.metastore_uri) .await .unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 1); let index_path = test_env.indexes_dir_path.join(&test_env.index_id); @@ -764,30 +794,54 @@ async fn test_garbage_collect_index_cli() { garbage_collect_index_cli(args).await.unwrap(); // Split should still exists within grace period. - let metastore = refresh_metastore(metastore).await.unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.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_ids = [split.split_metadata.split_id.as_str()]; metastore - .mark_splits_for_deletion(index_uid.clone(), &split_ids) + .mark_splits_for_deletion(MarkSplitsForDeletionRequest::new( + index_uid.to_string(), + vec![split.split_metadata.split_id.to_string()], + )) .await .unwrap(); metastore - .delete_splits(index_uid.clone(), &split_ids) + .delete_splits(DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: splits + .iter() + .map(|split| split.split_metadata.split_id.to_string()) + .collect(), + }) .await .unwrap(); metastore - .stage_splits(index_uid.clone(), vec![split.split_metadata]) + .stage_splits( + StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split.split_metadata.clone(), + ) + .unwrap(), + ) .await .unwrap(); assert_eq!(split_path.try_exists().unwrap(), true); - let metastore = refresh_metastore(metastore).await.unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits[0].split_state, SplitState::Staged); let args = create_gc_args(3600); @@ -796,8 +850,13 @@ async fn test_garbage_collect_index_cli() { assert_eq!(split_path.try_exists().unwrap(), true); // Staged splits should still exist within grace period. - let metastore = refresh_metastore(metastore).await.unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 1); assert_eq!(splits[0].split_state, SplitState::Staged); @@ -809,8 +868,13 @@ async fn test_garbage_collect_index_cli() { garbage_collect_index_cli(args).await.unwrap(); - let metastore = refresh_metastore(metastore).await.unwrap(); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let mut metastore = refresh_metastore(metastore).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); // Splits should be deleted from both metastore and file system. assert_eq!(splits.len(), 0); assert_eq!(split_path.try_exists().unwrap(), false); diff --git a/quickwit/quickwit-cli/tests/helpers.rs b/quickwit/quickwit-cli/tests/helpers.rs index c05f0e3d57a..5c08dafbdc7 100644 --- a/quickwit/quickwit-cli/tests/helpers.rs +++ b/quickwit/quickwit-cli/tests/helpers.rs @@ -29,7 +29,8 @@ use quickwit_common::net::find_available_tcp_port; use quickwit_common::test_utils::wait_for_server_ready; use quickwit_common::uri::Uri; use quickwit_config::service::QuickwitService; -use quickwit_metastore::{IndexMetadata, Metastore, MetastoreResolver}; +use quickwit_metastore::{IndexMetadata, IndexMetadataResponseExt, MetastoreResolver}; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_storage::{Storage, StorageResolver}; use reqwest::Url; use tempfile::{tempdir, TempDir}; @@ -111,7 +112,7 @@ pub struct TestEnv { /// The metastore URI. pub metastore_uri: Uri, pub metastore_resolver: MetastoreResolver, - pub metastore: Arc, + pub metastore: MetastoreServiceClient, pub config_uri: Uri, pub cluster_endpoint: Url, pub index_config_uri: Uri, @@ -125,7 +126,7 @@ pub struct TestEnv { impl TestEnv { // For cache reason, it's safer to always create an instance and then make your assertions. - pub async fn metastore(&self) -> Arc { + pub async fn metastore(&self) -> MetastoreServiceClient { self.metastore_resolver .resolve(&self.metastore_uri) .await @@ -142,8 +143,9 @@ impl TestEnv { let index_metadata = self .metastore() .await - .index_metadata(&self.index_id) - .await?; + .index_metadata(IndexMetadataRequest::for_index_id(self.index_id.clone())) + .await? + .deserialize_index_metadata()?; Ok(index_metadata) } diff --git a/quickwit/quickwit-codegen/Cargo.toml b/quickwit/quickwit-codegen/Cargo.toml index b4566200ef7..daa828f6fc7 100644 --- a/quickwit/quickwit-codegen/Cargo.toml +++ b/quickwit/quickwit-codegen/Cargo.toml @@ -21,4 +21,5 @@ syn = { workspace = true } tonic-build = { workspace = true } [dev-dependencies] +futures = { workspace = true } serde = { workspace = true } diff --git a/quickwit/quickwit-codegen/example/Cargo.toml b/quickwit/quickwit-codegen/example/Cargo.toml index 175cbb9881f..16c0f35825c 100644 --- a/quickwit/quickwit-codegen/example/Cargo.toml +++ b/quickwit/quickwit-codegen/example/Cargo.toml @@ -27,6 +27,7 @@ utoipa = { workspace = true } quickwit-actors = { workspace = true } quickwit-common = { workspace = true } +quickwit-macros = { workspace = true } [dev-dependencies] mockall = { workspace = true} diff --git a/quickwit/quickwit-codegen/example/build.rs b/quickwit/quickwit-codegen/example/build.rs index 37a822a5745..b6b0b49f534 100644 --- a/quickwit/quickwit-codegen/example/build.rs +++ b/quickwit/quickwit-codegen/example/build.rs @@ -26,6 +26,7 @@ fn main() { "crate::HelloResult", "crate::HelloError", true, + true, &[], ) .unwrap(); diff --git a/quickwit/quickwit-codegen/example/src/codegen/hello.rs b/quickwit/quickwit-codegen/example/src/codegen/hello.rs index 0ddc02277eb..8916d34687d 100644 --- a/quickwit/quickwit-codegen/example/src/codegen/hello.rs +++ b/quickwit/quickwit-codegen/example/src/codegen/hello.rs @@ -42,6 +42,17 @@ pub struct PingResponse { } /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; +use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; +impl PrometheusLabels<1> for HelloRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("hello")]) + } +} +impl PrometheusLabels<1> for GoodbyeRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("goodbye")]) + } +} pub type HelloStream = quickwit_common::ServiceStream>; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] @@ -80,6 +91,11 @@ impl HelloClient { where T: Hello, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < MockHello > (), + "`MockHello` must be wrapped in a `MockHelloWrapper`. Use `MockHello::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( @@ -184,7 +200,7 @@ pub mod hello_mock { self.inner.lock().await.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.blocking_lock().endpoints() + futures::executor::block_on(self.inner.lock()).endpoints() } } impl From for HelloClient { diff --git a/quickwit/quickwit-codegen/src/codegen.rs b/quickwit/quickwit-codegen/src/codegen.rs index 8fa72c2875a..943ef916cbb 100644 --- a/quickwit/quickwit-codegen/src/codegen.rs +++ b/quickwit/quickwit-codegen/src/codegen.rs @@ -17,6 +17,8 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::collections::HashSet; + use heck::ToSnakeCase; use proc_macro2::TokenStream; use prost_build::{Comments, Method, Service, ServiceGenerator}; @@ -34,6 +36,7 @@ impl Codegen { result_type_path: &str, error_type_path: &str, generate_extra_service_methods: bool, + generate_prom_labels_for_requests: bool, includes: &[&str], ) -> anyhow::Result<()> { Self::run_with_config( @@ -42,17 +45,20 @@ impl Codegen { result_type_path, error_type_path, generate_extra_service_methods, + generate_prom_labels_for_requests, includes, ProstConfig::default(), ) } + #[allow(clippy::too_many_arguments)] pub fn run_with_config( protos: &[&str], out_dir: &str, result_type_path: &str, error_type_path: &str, generate_extra_service_methods: bool, + generate_prom_label_for_requests: bool, includes: &[&str], mut prost_config: ProstConfig, ) -> anyhow::Result<()> { @@ -60,6 +66,7 @@ impl Codegen { result_type_path, error_type_path, generate_extra_service_methods, + generate_prom_label_for_requests, )); prost_config @@ -88,6 +95,7 @@ struct QuickwitServiceGenerator { result_type_path: String, error_type_path: String, generate_extra_service_methods: bool, + generate_prom_labels_for_requests: bool, inner: Box, } @@ -96,6 +104,7 @@ impl QuickwitServiceGenerator { result_type_path: &str, error_type_path: &str, generate_extra_service_methods: bool, + generate_prom_labels_for_requests: bool, ) -> Self { let inner = Box::new(WithSuffixServiceGenerator::new( "Grpc", @@ -105,6 +114,7 @@ impl QuickwitServiceGenerator { result_type_path: result_type_path.to_string(), error_type_path: error_type_path.to_string(), generate_extra_service_methods, + generate_prom_labels_for_requests, inner, } } @@ -117,6 +127,7 @@ impl ServiceGenerator for QuickwitServiceGenerator { &self.result_type_path, &self.error_type_path, self.generate_extra_service_methods, + self.generate_prom_labels_for_requests, ); let ast: syn::File = syn::parse2(tokens).expect("Tokenstream should be a valid Syn AST."); let pretty_code = prettyplease::unparse(&ast); @@ -229,6 +240,7 @@ fn generate_all( result_type_path: &str, error_type_path: &str, generate_extra_service_methods: bool, + implement_prom_labels_for_requests: bool, ) -> TokenStream { let context = CodegenContext::from_service( service, @@ -245,12 +257,17 @@ fn generate_all( let tower_mailbox = generate_tower_mailbox(&context); let grpc_client_adapter = generate_grpc_client_adapter(&context); let grpc_server_adapter = generate_grpc_server_adapter(&context); + let prom_labels_impl = if implement_prom_labels_for_requests { + generate_prom_labels_impl_for_requests(&context) + } else { + TokenStream::new() + }; quote! { // The line below is necessary to opt out of the license header check. /// BEGIN quickwit-codegen - use tower::{Layer, Service, ServiceExt}; + #prom_labels_impl #stream_type_alias @@ -285,6 +302,17 @@ struct SynMethod { } impl SynMethod { + fn request_prom_label(&self) -> String { + self.request_type + .segments + .last() + .unwrap() + .ident + .to_string() + .trim_end_matches("Request") + .to_snake_case() + } + fn request_type(&self, mock: bool) -> TokenStream { let request_type = if mock { let request_type = &self.request_type; @@ -339,6 +367,35 @@ impl SynMethod { } } +fn generate_prom_labels_impl_for_requests(context: &CodegenContext) -> TokenStream { + let mut stream = TokenStream::new(); + stream.extend(quote! { + use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; + }); + let mut implemented_request_types: HashSet = HashSet::new(); + for syn_method in &context.methods { + if syn_method.client_streaming { + continue; + } + let request_type = syn_method.request_type(false); + let request_type_snake_case = syn_method.request_prom_label(); + if implemented_request_types.contains(&request_type_snake_case) { + continue; + } else { + implemented_request_types.insert(request_type_snake_case.clone()); + let method = quote! { + impl PrometheusLabels<1> for #request_type { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed(#request_type_snake_case),]) + } + } + }; + stream.extend(method); + } + } + stream +} + fn generate_comment_attributes(comments: &Comments) -> Vec { let mut attributes = Vec::with_capacity(comments.leading.len()); @@ -434,7 +491,10 @@ fn generate_client(context: &CodegenContext) -> TokenStream { let tower_block_builder_name = &context.tower_block_builder_name; let mock_name = &context.mock_name; let mock_wrapper_name = quote::format_ident!("{}Wrapper", mock_name); - + let error_mesage = format!( + "`{}` must be wrapped in a `{}`. Use `{}::from(mock)` to instantiate the client.", + mock_name, mock_wrapper_name, mock_name + ); let additional_client_methods = if context.generate_extra_service_methods { generate_additional_methods_calling_inner() } else { @@ -447,7 +507,7 @@ fn generate_client(context: &CodegenContext) -> TokenStream { } fn endpoints(&self) -> Vec { - self.inner.blocking_lock().endpoints() + futures::executor::block_on(self.inner.lock()).endpoints() } } } else { @@ -465,6 +525,8 @@ fn generate_client(context: &CodegenContext) -> TokenStream { where T: #service_name, { + #[cfg(any(test, feature = "testsuite"))] + assert!(std::any::TypeId::of::() != std::any::TypeId::of::<#mock_name>(), #error_mesage); Self { inner: Box::new(instance), } diff --git a/quickwit/quickwit-common/src/tower/event_listener.rs b/quickwit/quickwit-common/src/tower/event_listener.rs index 30d8a0ab978..1865d8c55ad 100644 --- a/quickwit/quickwit-common/src/tower/event_listener.rs +++ b/quickwit/quickwit-common/src/tower/event_listener.rs @@ -26,6 +26,7 @@ use tower::{Layer, Service}; use crate::pubsub::{Event, EventBroker}; +#[derive(Clone)] pub struct EventListener { inner: S, event_broker: EventBroker, diff --git a/quickwit/quickwit-common/src/tower/mod.rs b/quickwit/quickwit-common/src/tower/mod.rs index 1aacd7fec8c..4b429442291 100644 --- a/quickwit/quickwit-common/src/tower/mod.rs +++ b/quickwit/quickwit-common/src/tower/mod.rs @@ -46,6 +46,7 @@ pub use pool::Pool; pub use rate::{ConstantRate, Rate}; pub use rate_estimator::{RateEstimator, SmaRateEstimator}; pub use rate_limit::{RateLimit, RateLimitLayer}; +pub use retry::{RetryLayer, RetryPolicy}; pub use transport::{make_channel, warmup_channel, BalanceChannel}; pub type BoxError = Box; diff --git a/quickwit/quickwit-common/src/tower/retry.rs b/quickwit/quickwit-common/src/tower/retry.rs index b41d4169bc8..dadfebc6197 100644 --- a/quickwit/quickwit-common/src/tower/retry.rs +++ b/quickwit/quickwit-common/src/tower/retry.rs @@ -25,12 +25,38 @@ use std::task::{Context, Poll}; use futures::Future; use pin_project::pin_project; use tokio::time::Sleep; -use tower::retry::Policy; +use tower::retry::{Policy, Retry}; +use tower::Layer; use tracing::debug; use crate::retry::{RetryParams, Retryable}; -#[derive(Debug, Clone, Copy, Default)] +/// Retry layer copy/pasted from `tower::retry::RetryLayer` +/// but which implements `Clone`. +impl Layer for RetryLayer

+where P: Clone +{ + type Service = Retry; + + fn layer(&self, service: S) -> Self::Service { + let policy = self.policy.clone(); + Retry::new(policy, service) + } +} + +#[derive(Clone, Debug)] +pub struct RetryLayer

{ + policy: P, +} + +impl

RetryLayer

{ + /// Create a new [`RetryLayer`] from a retry policy + pub fn new(policy: P) -> Self { + RetryLayer { policy } + } +} + +#[derive(Clone, Copy, Debug, Default)] pub struct RetryPolicy { num_attempts: usize, retry_params: RetryParams, @@ -110,7 +136,6 @@ mod tests { use std::sync::{Arc, Mutex}; use futures::future::{ready, Ready}; - use tower::retry::RetryLayer; use tower::{Layer, Service, ServiceExt}; use super::*; diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 7a5e6ff4793..824b18bf5a2 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; use std::time::Duration; use anyhow::Context; @@ -25,9 +24,9 @@ use async_trait::async_trait; use quickwit_actors::{ Actor, ActorContext, ActorExitStatus, ActorHandle, Handler, Mailbox, Supervisor, Universe, }; -use quickwit_config::{IndexConfig, SourceConfig}; +use quickwit_config::SourceConfig; use quickwit_ingest::IngesterPool; -use quickwit_metastore::{IndexMetadata, Metastore}; +use quickwit_metastore::IndexMetadata; use quickwit_proto::control_plane::{ ControlPlaneError, ControlPlaneResult, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, @@ -35,7 +34,7 @@ use quickwit_proto::control_plane::{ use quickwit_proto::metastore::{ serde_utils as metastore_serde_utils, AddSourceRequest, CloseShardsRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteShardsRequest, DeleteSourceRequest, - EmptyResponse, MetastoreError, ToggleSourceRequest, + EmptyResponse, MetastoreError, MetastoreService, MetastoreServiceClient, ToggleSourceRequest, }; use quickwit_proto::{IndexUid, NodeId}; use serde::Serialize; @@ -58,9 +57,8 @@ struct ControlPlanLoop; #[derive(Debug)] pub struct ControlPlane { - metastore: Arc, + metastore: MetastoreServiceClient, model: ControlPlaneModel, - // The control plane state is split into to independent functions, that we naturally isolated // code wise and state wise. // @@ -79,7 +77,7 @@ impl ControlPlane { self_node_id: NodeId, indexer_pool: IndexerPool, ingester_pool: IngesterPool, - metastore: Arc, + metastore: MetastoreServiceClient, replication_factor: usize, ) -> (Mailbox, ActorHandle>) { universe.spawn_builder().supervise_fn(move || { @@ -124,7 +122,7 @@ impl Actor for ControlPlane { async fn initialize(&mut self, ctx: &ActorContext) -> Result<(), ActorExitStatus> { self.model - .load_from_metastore(&*self.metastore, ctx.progress()) + .load_from_metastore(&mut self.metastore, ctx.progress()) .await .context("failed to intialize the model")?; @@ -138,7 +136,6 @@ impl Actor for ControlPlane { ctx.schedule_self_msg(CONTROL_PLAN_LOOP_INTERVAL, ControlPlanLoop) .await; - Ok(()) } } @@ -214,20 +211,16 @@ impl Handler for ControlPlane { request: CreateIndexRequest, _ctx: &ActorContext, ) -> Result { - let index_config: IndexConfig = - match metastore_serde_utils::from_json_str(&request.index_config_json) { - Ok(index_config) => index_config, - Err(error) => { - return Ok(Err(ControlPlaneError::from(error))); - } - }; - - let index_uid = match self.metastore.create_index(index_config.clone()).await { - Ok(index_uid) => index_uid, - Err(metastore_error) => { - return convert_metastore_error(metastore_error); + let index_config = match metastore_serde_utils::from_json_str(&request.index_config_json) { + Ok(index_config) => index_config, + Err(error) => { + return Ok(Err(ControlPlaneError::from(error))); } }; + let index_uid: IndexUid = match self.metastore.create_index(request).await { + Ok(response) => response.index_uid.into(), + Err(metastore_error) => return convert_metastore_error(metastore_error), + }; let index_metadata: IndexMetadata = IndexMetadata::new_with_index_uid(index_uid.clone(), index_config); @@ -253,10 +246,10 @@ impl Handler for ControlPlane { request: DeleteIndexRequest, _ctx: &ActorContext, ) -> Result { - let index_uid: IndexUid = request.index_uid.into(); + let index_uid: IndexUid = request.index_uid.clone().into(); - if let Err(metastore_error) = self.metastore.delete_index(index_uid.clone()).await { - return convert_metastore_error(metastore_error); + if let Err(error) = self.metastore.delete_index(request).await { + return Ok(Err(ControlPlaneError::from(error))); }; self.model.delete_index(&index_uid); @@ -282,7 +275,7 @@ impl Handler for ControlPlane { request: AddSourceRequest, _ctx: &ActorContext, ) -> Result { - let index_uid: IndexUid = request.index_uid.into(); + let index_uid: IndexUid = request.index_uid.clone().into(); let source_config: SourceConfig = match metastore_serde_utils::from_json_str(&request.source_config_json) { Ok(source_config) => source_config, @@ -290,13 +283,8 @@ impl Handler for ControlPlane { return Ok(Err(ControlPlaneError::from(error))); } }; - - if let Err(metastore_error) = self - .metastore - .add_source(index_uid.clone(), source_config.clone()) - .await - { - return convert_metastore_error(metastore_error); + if let Err(error) = self.metastore.add_source(request).await { + return Ok(Err(ControlPlaneError::from(error))); }; self.model @@ -323,18 +311,11 @@ impl Handler for ControlPlane { request: ToggleSourceRequest, _ctx: &ActorContext, ) -> Result { - let index_uid: IndexUid = request.index_uid.into(); - - if let Err(metastore_error) = self - .metastore - .toggle_source(index_uid.clone(), &request.source_id, request.enable) - .await - { - return convert_metastore_error(metastore_error); - } + if let Err(error) = self.metastore.toggle_source(request).await { + return Ok(Err(ControlPlaneError::from(error))); + }; // TODO update the internal view. - // TODO: Refine the event. Notify index will have the effect to reload the entire state from // the metastore. We should update the state of the control plane. self.indexing_scheduler.on_index_change(&self.model).await?; @@ -355,17 +336,12 @@ impl Handler for ControlPlane { request: DeleteSourceRequest, _ctx: &ActorContext, ) -> Result { - let index_uid: IndexUid = request.index_uid.into(); - - if let Err(metastore_error) = self - .metastore - .delete_source(index_uid.clone(), &request.source_id) - .await - { + let index_uid: IndexUid = request.index_uid.clone().into(); + let source_id = request.source_id.clone(); + if let Err(metastore_error) = self.metastore.delete_source(request).await { return convert_metastore_error(metastore_error); - } - - self.model.delete_source(&index_uid, &request.source_id); + }; + self.model.delete_source(&index_uid, &source_id); self.indexing_scheduler.on_index_change(&self.model).await?; let response = EmptyResponse {}; Ok(Ok(response)) @@ -436,12 +412,16 @@ impl Handler for ControlPlane { #[cfg(test)] mod tests { use quickwit_actors::{AskError, Observe, SupervisorMetrics}; - use quickwit_config::{SourceParams, INGEST_SOURCE_ID}; - use quickwit_metastore::{IndexMetadata, ListIndexesQuery, MockMetastore}; + use quickwit_config::{IndexConfig, SourceParams, INGEST_SOURCE_ID}; + use quickwit_metastore::{ + CreateIndexRequestExt, IndexMetadata, ListIndexesMetadataRequestExt, + ListIndexesMetadataResponseExt, + }; use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; use quickwit_proto::ingest::Shard; use quickwit_proto::metastore::{ - EntityKind, ListShardsRequest, ListShardsResponse, ListShardsSubresponse, SourceType, + EntityKind, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, + ListShardsResponse, ListShardsSubresponse, MetastoreError, SourceType, }; use super::*; @@ -455,20 +435,26 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_create_index() - .returning(|index_config| { + .withf(|create_index_request| { + let index_config: IndexConfig = + serde_json::from_str(&create_index_request.index_config_json).unwrap(); assert_eq!(index_config.index_id, "test-index"); assert_eq!(index_config.index_uri, "ram:///test-index"); - - let index_uid: IndexUid = "test-index:0".into(); - Ok(index_uid) + true + }) + .returning(|_| { + Ok(CreateIndexResponse { + index_uid: "test-index:0".to_string(), + }) }); mock_metastore - .expect_list_indexes_metadatas() - .returning(|_| Ok(Vec::new())); - let metastore = Arc::new(mock_metastore); + .expect_list_indexes_metadata() + .returning(|_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(Vec::new()).unwrap()) + }); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -477,7 +463,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let index_config = IndexConfig::for_test("test-index", "ram:///test-index"); @@ -504,15 +490,16 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); - mock_metastore.expect_delete_index().returning(|index_uid| { - assert_eq!(index_uid.as_str(), "test-index:0"); - Ok(()) - }); + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore + .expect_delete_index() + .withf(|delete_index_request| delete_index_request.index_uid == "test-index:0") + .returning(|_| Ok(EmptyResponse {})); mock_metastore - .expect_list_indexes_metadatas() - .returning(|_| Ok(Vec::new())); - let metastore = Arc::new(mock_metastore); + .expect_list_indexes_metadata() + .returning(|_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(Vec::new()).unwrap()) + }); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -521,7 +508,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let delete_index_request = DeleteIndexRequest { @@ -539,6 +526,7 @@ mod tests { #[tokio::test] async fn test_control_plane_add_source() { + quickwit_common::setup_logging_for_tests(); let universe = Universe::with_accelerated_time(); let cluster_id = "test-cluster".to_string(); @@ -548,20 +536,25 @@ mod tests { let index_metadata = IndexMetadata::for_test("test-index", "ram://test"); let index_uid = index_metadata.index_uid.clone(); - - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_add_source() - .returning(|index_uid, source_config| { - assert_eq!(index_uid.as_str(), "test-index:0"); + .withf(|add_source_request| { + let source_config: SourceConfig = + serde_json::from_str(&add_source_request.source_config_json).unwrap(); assert_eq!(source_config.source_id, "test-source"); assert_eq!(source_config.source_type(), SourceType::Void); - Ok(()) - }); + true + }) + .returning(|_| Ok(EmptyResponse {})); mock_metastore - .expect_list_indexes_metadatas() - .returning(move |_| Ok(vec![index_metadata.clone()])); - let metastore = Arc::new(mock_metastore); + .expect_list_indexes_metadata() + .returning(move |_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -570,7 +563,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let source_config = SourceConfig::for_test("test-source", SourceParams::void()); @@ -597,19 +590,21 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_toggle_source() - .returning(|index_uid, source_id, enable| { - assert_eq!(index_uid.as_str(), "test-index:0"); - assert_eq!(source_id, "test-source"); - assert!(enable); - Ok(()) - }); + .withf(|toggle_source_request| { + assert_eq!(toggle_source_request.index_uid, "test-index:0"); + assert_eq!(toggle_source_request.source_id, "test-source"); + assert!(toggle_source_request.enable); + true + }) + .returning(|_| Ok(EmptyResponse {})); mock_metastore - .expect_list_indexes_metadatas() - .returning(|_| Ok(Vec::new())); - let metastore = Arc::new(mock_metastore); + .expect_list_indexes_metadata() + .returning(|_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(Vec::new()).unwrap()) + }); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -618,7 +613,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let toggle_source_request = ToggleSourceRequest { @@ -645,18 +640,20 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_delete_source() - .returning(|index_uid, source_id| { - assert_eq!(index_uid.as_str(), "test-index:0"); - assert_eq!(source_id, "test-source"); - Ok(()) - }); + .withf(|delete_source_request| { + assert_eq!(delete_source_request.index_uid, "test-index:0"); + assert_eq!(delete_source_request.source_id, "test-source"); + true + }) + .returning(|_| Ok(EmptyResponse {})); mock_metastore - .expect_list_indexes_metadatas() - .returning(|_| Ok(Vec::new())); - let metastore = Arc::new(mock_metastore); + .expect_list_indexes_metadata() + .returning(|_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(Vec::new()).unwrap()) + }); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -665,7 +662,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let delete_source_request = DeleteSourceRequest { @@ -691,14 +688,17 @@ mod tests { let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .returning(|_| { let mut index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let source_config = SourceConfig::for_test(INGEST_SOURCE_ID, SourceParams::void()); index_metadata.add_source(source_config).unwrap(); - Ok(vec![index_metadata]) + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(vec![index_metadata]) + .unwrap(), + ) }); mock_metastore.expect_list_shards().returning(|request| { assert_eq!(request.subrequests.len(), 1); @@ -721,7 +721,6 @@ mod tests { let response = ListShardsResponse { subresponses }; Ok(response) }); - let metastore = Arc::new(mock_metastore); let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -730,7 +729,7 @@ mod tests { self_node_id, indexer_pool, ingester_pool, - metastore, + MetastoreServiceClient::from(mock_metastore), replication_factor, ); let get_open_shards_request = GetOrCreateOpenShardsRequest { @@ -767,18 +766,18 @@ mod tests { let node_id = NodeId::new("test_node".to_string()); let indexer_pool = IndexerPool::default(); let ingester_pool = IngesterPool::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); let source = SourceConfig::ingest_default(); index_0.add_source(source.clone()).unwrap(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(2) // 1 for the first initialization, 1 after the respawn of the control plane. - .returning(|list_indexes_query: ListIndexesQuery| { - assert!(matches!(&list_indexes_query, &ListIndexesQuery::All)); - Ok(Vec::new()) + .returning(|list_indexes_request: ListIndexesMetadataRequest| { + assert_eq!(list_indexes_request, ListIndexesMetadataRequest::all()); + Ok(ListIndexesMetadataResponse::empty()) }); mock_metastore.expect_list_shards().return_once( |_list_shards_request: ListShardsRequest| { @@ -789,27 +788,31 @@ mod tests { }, ); let index_uid = IndexUid::new("test-index"); - let index_uid_clone = index_uid.clone(); - mock_metastore - .expect_create_index() - .times(1) - .return_once(|_index_config: IndexConfig| Ok(index_uid_clone)); - mock_metastore - .expect_create_index() - .times(1) - .return_once(|index_config: IndexConfig| { + let index_uid_string = index_uid.to_string(); + mock_metastore.expect_create_index().times(1).return_once( + |_create_index_request: CreateIndexRequest| { + Ok(CreateIndexResponse { + index_uid: index_uid_string, + }) + }, + ); + mock_metastore.expect_create_index().times(1).return_once( + |create_index_request: CreateIndexRequest| { Err(MetastoreError::AlreadyExists(EntityKind::Index { - index_id: index_config.index_id, + index_id: create_index_request + .deserialize_index_config() + .unwrap() + .index_id, })) - }); - mock_metastore - .expect_create_index() - .times(1) - .return_once(|_index_config: IndexConfig| { + }, + ); + mock_metastore.expect_create_index().times(1).return_once( + |_create_index_request: CreateIndexRequest| { Err(MetastoreError::Connection { message: "Fake connection error.".to_string(), }) - }); + }, + ); let (control_plane_mailbox, control_plane_handle) = ControlPlane::spawn( &universe, @@ -817,7 +820,7 @@ mod tests { node_id, indexer_pool, ingester_pool, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), 1, ); diff --git a/quickwit/quickwit-control-plane/src/control_plane_model.rs b/quickwit/quickwit-control-plane/src/control_plane_model.rs index 93fd3648721..60f10f92dbd 100644 --- a/quickwit/quickwit-control-plane/src/control_plane_model.rs +++ b/quickwit/quickwit-control-plane/src/control_plane_model.rs @@ -25,10 +25,15 @@ use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use quickwit_common::Progress; use quickwit_config::{SourceConfig, INGEST_SOURCE_ID}; -use quickwit_metastore::{IndexMetadata, ListIndexesQuery, Metastore}; +use quickwit_metastore::{ + IndexMetadata, ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, +}; use quickwit_proto::control_plane::ControlPlaneResult; use quickwit_proto::ingest::{Shard, ShardState}; -use quickwit_proto::metastore::{EntityKind, ListShardsSubrequest, MetastoreError}; +use quickwit_proto::metastore::{ + EntityKind, ListIndexesMetadataRequest, ListShardsSubrequest, MetastoreError, MetastoreService, + MetastoreServiceClient, +}; use quickwit_proto::types::IndexId; use quickwit_proto::{metastore, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; use serde::Serialize; @@ -108,15 +113,15 @@ impl ControlPlaneModel { pub async fn load_from_metastore( &mut self, - metastore: &dyn Metastore, + metastore: &mut MetastoreServiceClient, progress: &Progress, ) -> ControlPlaneResult<()> { let now = Instant::now(); self.clear(); let index_metadatas = progress - .protect_future(metastore.list_indexes_metadatas(ListIndexesQuery::All)) - .await?; - + .protect_future(metastore.list_indexes_metadata(ListIndexesMetadataRequest::all())) + .await? + .deserialize_indexes_metadata()?; let num_indexes = index_metadatas.len(); self.index_table.reserve(num_indexes); @@ -461,8 +466,9 @@ impl ShardTable { #[cfg(test)] mod tests { use quickwit_config::SourceConfig; - use quickwit_metastore::{IndexMetadata, MockMetastore}; + use quickwit_metastore::IndexMetadata; use quickwit_proto::ingest::Shard; + use quickwit_proto::metastore::ListIndexesMetadataResponse; use super::*; @@ -677,11 +683,11 @@ mod tests { async fn test_control_plane_model_load_shard_table() { let progress = Progress::default(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() - .returning(|query| { - assert!(matches!(query, ListIndexesQuery::All)); + .expect_list_indexes_metadata() + .returning(|request| { + assert_eq!(request, ListIndexesMetadataRequest::all()); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); let source = SourceConfig::ingest_default(); @@ -691,7 +697,7 @@ mod tests { index_1.add_source(source).unwrap(); let indexes = vec![index_0, index_1]; - Ok(indexes) + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(indexes).unwrap()) }); mock_metastore.expect_list_shards().returning(|request| { assert_eq!(request.subrequests.len(), 2); @@ -725,8 +731,9 @@ mod tests { Ok(response) }); let mut model = ControlPlaneModel::default(); + let mut metastore_client = MetastoreServiceClient::from(mock_metastore); model - .load_from_metastore(&mock_metastore, &progress) + .load_from_metastore(&mut metastore_client, &progress) .await .unwrap(); diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs index 468cba7e76e..31f07574d0f 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs @@ -19,14 +19,13 @@ use std::cmp::Ordering; use std::fmt; -use std::sync::Arc; use std::time::{Duration, Instant}; use anyhow::Context; use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; -use quickwit_metastore::Metastore; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingService, IndexingTask}; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::NodeId; use serde::Serialize; use tracing::{debug, error, info, warn}; @@ -96,7 +95,7 @@ pub struct IndexingSchedulerState { pub struct IndexingScheduler { cluster_id: String, self_node_id: NodeId, - metastore: Arc, + metastore: MetastoreServiceClient, indexer_pool: IndexerPool, state: IndexingSchedulerState, } @@ -106,7 +105,7 @@ impl fmt::Debug for IndexingScheduler { f.debug_struct("IndexingScheduler") .field("cluster_id", &self.cluster_id) .field("node_id", &self.self_node_id) - .field("metastore_uri", &self.metastore.uri()) + .field("metastore", &self.metastore) .field( "last_applied_plan_ts", &self.state.last_applied_plan_timestamp, @@ -119,7 +118,7 @@ impl IndexingScheduler { pub fn new( cluster_id: String, self_node_id: NodeId, - metastore: Arc, + metastore: MetastoreServiceClient, indexer_pool: IndexerPool, ) -> Self { Self { diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 5e94de90854..7ec8524dfd8 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -18,21 +18,21 @@ // along with this program. If not, see . use std::fmt; -use std::sync::Arc; use std::time::Duration; use fnv::FnvHashSet; use itertools::Itertools; use quickwit_common::Progress; use quickwit_ingest::IngesterPool; -use quickwit_metastore::Metastore; use quickwit_proto::control_plane::{ ControlPlaneError, ControlPlaneResult, GetOpenShardsSubresponse, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, }; use quickwit_proto::ingest::ingester::{IngesterService, PingRequest}; use quickwit_proto::ingest::IngestV2Error; -use quickwit_proto::metastore::{EntityKind, MetastoreError}; +use quickwit_proto::metastore::{ + EntityKind, MetastoreError, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::types::NodeId; use quickwit_proto::{metastore, IndexUid}; use rand::seq::SliceRandom; @@ -47,7 +47,7 @@ const PING_LEADER_TIMEOUT: Duration = if cfg!(test) { }; pub struct IngestController { - metastore: Arc, + metastore: MetastoreServiceClient, ingester_pool: IngesterPool, replication_factor: usize, } @@ -64,7 +64,7 @@ impl fmt::Debug for IngestController { impl IngestController { pub fn new( - metastore: Arc, + metastore: MetastoreServiceClient, ingester_pool: IngesterPool, replication_factor: usize, ) -> Self { @@ -284,7 +284,7 @@ pub enum PingError { mod tests { use quickwit_config::{SourceConfig, SourceParams}; - use quickwit_metastore::{IndexMetadata, MockMetastore}; + use quickwit_metastore::IndexMetadata; use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; use quickwit_proto::ingest::ingester::{ IngesterServiceClient, MockIngesterService, PingResponse, @@ -297,12 +297,14 @@ mod tests { async fn test_ingest_controller_ping_leader() { let progress = Progress::default(); - let mock_metastore = MockMetastore::default(); - let metastore = Arc::new(mock_metastore); + let mock_metastore = MetastoreServiceClient::mock(); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool.clone(), replication_factor); + let mut ingest_controller = IngestController::new( + MetastoreServiceClient::from(mock_metastore), + ingester_pool.clone(), + replication_factor, + ); let leader_id: NodeId = "test-ingester-0".into(); let error = ingest_controller @@ -370,12 +372,14 @@ mod tests { async fn test_ingest_controller_find_leader_replication_factor_1() { let progress = Progress::default(); - let mock_metastore = MockMetastore::default(); - let metastore = Arc::new(mock_metastore); + let mock_metastore = MetastoreServiceClient::mock(); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool.clone(), replication_factor); + let mut ingest_controller = IngestController::new( + MetastoreServiceClient::from(mock_metastore), + ingester_pool.clone(), + replication_factor, + ); let leader_follower_pair = ingest_controller .find_leader_and_follower(&mut FnvHashSet::default(), &progress) @@ -419,12 +423,14 @@ mod tests { async fn test_ingest_controller_find_leader_replication_factor_2() { let progress = Progress::default(); - let mock_metastore = MockMetastore::default(); - let metastore = Arc::new(mock_metastore); + let mock_metastore = MetastoreServiceClient::mock(); let ingester_pool = IngesterPool::default(); let replication_factor = 2; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool.clone(), replication_factor); + let mut ingest_controller = IngestController::new( + MetastoreServiceClient::from(mock_metastore), + ingester_pool.clone(), + replication_factor, + ); let leader_follower_pair = ingest_controller .find_leader_and_follower(&mut FnvHashSet::default(), &progress) @@ -504,7 +510,7 @@ mod tests { let progress = Progress::default(); let index_uid_1_str_clone = index_uid_1_str.clone(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_open_shards() .once() @@ -526,7 +532,6 @@ mod tests { let response = metastore::OpenShardsResponse { subresponses }; Ok(response) }); - let metastore = Arc::new(mock_metastore); let ingester_pool = IngesterPool::default(); let mut mock_ingester = MockIngesterService::default(); @@ -544,8 +549,11 @@ mod tests { ingester_pool.insert("test-ingester-2".into(), ingester.clone()); let replication_factor = 2; - let mut ingest_controller = - IngestController::new(metastore, ingester_pool.clone(), replication_factor); + let mut ingest_controller = IngestController::new( + MetastoreServiceClient::from(mock_metastore), + ingester_pool.clone(), + replication_factor, + ); let mut model = ControlPlaneModel::default(); diff --git a/quickwit/quickwit-control-plane/src/tests.rs b/quickwit/quickwit-control-plane/src/tests.rs index 4873cbf0483..ec03c38a3f1 100644 --- a/quickwit/quickwit-control-plane/src/tests.rs +++ b/quickwit/quickwit-control-plane/src/tests.rs @@ -18,7 +18,6 @@ // along with this program. If not, see . use std::num::NonZeroUsize; -use std::sync::Arc; use std::time::Duration; use chitchat::transport::ChannelTransport; @@ -31,9 +30,11 @@ use quickwit_common::tower::{Change, Pool}; use quickwit_config::service::QuickwitService; use quickwit_config::{KafkaSourceParams, SourceConfig, SourceInputFormat, SourceParams}; use quickwit_indexing::IndexingService; -use quickwit_metastore::{IndexMetadata, ListIndexesQuery, MockMetastore}; +use quickwit_metastore::{IndexMetadata, ListIndexesMetadataResponseExt}; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingServiceClient}; -use quickwit_proto::metastore::ListShardsResponse; +use quickwit_proto::metastore::{ + ListIndexesMetadataResponse, ListShardsResponse, MetastoreServiceClient, +}; use quickwit_proto::NodeId; use serde_json::json; @@ -112,10 +113,11 @@ async fn start_control_plane( let index_metadata_1 = index_metadata_for_test(index_1, source_1, 2, 2); let mut index_metadata_2 = index_metadata_for_test(index_2, source_2, 1, 1); index_metadata_2.create_timestamp = index_metadata_1.create_timestamp + 1; - let mut metastore = MockMetastore::default(); - metastore.expect_list_indexes_metadatas().returning( - move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![index_metadata_2.clone(), index_metadata_1.clone()]) + let mut metastore = MetastoreServiceClient::mock(); + metastore.expect_list_indexes_metadata().returning( + move |_list_indexes_request: quickwit_proto::metastore::ListIndexesMetadataRequest| { + let indexes_metadata = vec![index_metadata_2.clone(), index_metadata_1.clone()]; + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata).unwrap()) }, ); metastore.expect_list_shards().returning(|_| { @@ -145,7 +147,7 @@ async fn start_control_plane( self_node_id, indexer_pool, ingester_pool, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), 1, ); diff --git a/quickwit/quickwit-index-management/Cargo.toml b/quickwit/quickwit-index-management/Cargo.toml index e10aca934a9..064fc509b7d 100644 --- a/quickwit/quickwit-index-management/Cargo.toml +++ b/quickwit/quickwit-index-management/Cargo.toml @@ -43,4 +43,5 @@ serde_yaml = { workspace = true } quickwit-common = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } quickwit-storage = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index 3e50a0d4ead..fc661fe3fc6 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -24,9 +24,15 @@ use std::time::Duration; use futures::Future; use quickwit_common::{PrettySample, Progress}; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitInfo, SplitMetadata, SplitState}; -use quickwit_proto::metastore::MetastoreError; -use quickwit_proto::IndexUid; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitInfo, SplitMetadata, + SplitState, +}; +use quickwit_proto::metastore::{ + DeleteSplitsRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, + MetastoreService, MetastoreServiceClient, +}; +use quickwit_proto::{IndexUid, SplitId}; use quickwit_storage::{BulkDeleteError, Storage}; use thiserror::Error; use time::OffsetDateTime; @@ -80,7 +86,7 @@ pub struct SplitRemovalInfo { pub async fn run_garbage_collect( index_uid: IndexUid, storage: Arc, - metastore: Arc, + mut metastore: MetastoreServiceClient, staged_grace_period: Duration, deletion_grace_period: Duration, dry_run: bool, @@ -94,23 +100,25 @@ pub async fn run_garbage_collect( .with_split_state(SplitState::Staged) .with_update_timestamp_lte(grace_period_timestamp); - let deletable_staged_splits: Vec = - protect_future(progress_opt, metastore.list_splits(query)) - .await? - .into_iter() - .map(|meta| meta.split_metadata) - .collect(); + let list_deletable_staged_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let deletable_staged_splits: Vec = protect_future( + progress_opt, + metastore.list_splits(list_deletable_staged_request), + ) + .await? + .deserialize_splits_metadata()?; if dry_run { - let query = ListSplitsQuery::for_index(index_uid.clone()) + let marked_for_deletion_query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_state(SplitState::MarkedForDeletion); - - let mut splits_marked_for_deletion = - protect_future(progress_opt, metastore.list_splits(query)) - .await? - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); + let marked_for_deletion_request = + ListSplitsRequest::try_from_list_splits_query(marked_for_deletion_query)?; + let mut splits_marked_for_deletion: Vec = protect_future( + progress_opt, + metastore.list_splits(marked_for_deletion_request), + ) + .await? + .deserialize_splits_metadata()?; splits_marked_for_deletion.extend(deletable_staged_splits); let candidate_entries: Vec = splits_marked_for_deletion @@ -124,14 +132,16 @@ pub async fn run_garbage_collect( } // Schedule all eligible staged splits for delete - let split_ids: Vec<&str> = deletable_staged_splits + let split_ids: Vec = deletable_staged_splits .iter() - .map(|split| split.split_id()) + .map(|split| split.split_id.to_string()) .collect(); if !split_ids.is_empty() { + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), split_ids); protect_future( progress_opt, - metastore.mark_splits_for_deletion(index_uid.clone(), &split_ids), + metastore.mark_splits_for_deletion(mark_splits_for_deletion_request), ) .await?; } @@ -162,7 +172,7 @@ async fn delete_splits_marked_for_deletion( index_uid: IndexUid, updated_before_timestamp: i64, storage: Arc, - metastore: Arc, + mut metastore: MetastoreServiceClient, progress_opt: Option<&Progress>, ) -> SplitRemovalInfo { let mut removed_splits = Vec::new(); @@ -174,19 +184,28 @@ async fn delete_splits_marked_for_deletion( .with_update_timestamp_lte(updated_before_timestamp) .with_limit(DELETE_SPLITS_BATCH_SIZE); - let list_splits_result = protect_future(progress_opt, metastore.list_splits(query)).await; + let list_splits_request = match ListSplitsRequest::try_from_list_splits_query(query) { + Ok(request) => request, + Err(error) => { + error!(error = ?error, "failed to build list splits request"); + 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_to_delete = match list_splits_result { - Ok(splits) => splits, + 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_to_delete = splits_to_delete - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); let num_splits_to_delete = splits_to_delete.len(); @@ -231,7 +250,7 @@ async fn delete_splits_marked_for_deletion( pub async fn delete_splits_from_storage_and_metastore( index_uid: IndexUid, storage: Arc, - metastore: Arc, + mut metastore: MetastoreServiceClient, splits: Vec, progress_opt: Option<&Progress>, ) -> anyhow::Result, DeleteSplitsError> { @@ -280,15 +299,16 @@ pub async fn delete_splits_from_storage_and_metastore( } }; if !successes.is_empty() { - let split_ids: Vec<&str> = successes + let split_ids: Vec = successes .iter() - .map(|split_info| split_info.split_id.as_str()) + .map(|split_info| split_info.split_id.to_string()) .collect(); - let metastore_result = protect_future( - progress_opt, - metastore.delete_splits(index_uid.clone(), &split_ids), - ) - .await; + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: split_ids.clone(), + }; + let metastore_result = + protect_future(progress_opt, metastore.delete_splits(delete_splits_request)).await; if let Err(metastore_error) = metastore_result { error!( @@ -327,9 +347,12 @@ mod tests { use itertools::Itertools; use quickwit_config::IndexConfig; use quickwit_metastore::{ - metastore_for_test, ListSplitsQuery, MockMetastore, SplitMetadata, SplitState, + metastore_for_test, CreateIndexRequestExt, ListSplitsQuery, SplitMetadata, SplitState, + StageSplitsRequestExt, + }; + use quickwit_proto::metastore::{ + CreateIndexRequest, EntityKind, ListSplitsResponse, StageSplitsRequest, }; - use quickwit_proto::metastore::EntityKind; use quickwit_proto::IndexUid; use quickwit_storage::{ storage_for_test, BulkDeleteError, DeleteFailure, MockStorage, PutPayload, @@ -341,12 +364,18 @@ mod tests { #[tokio::test] async fn test_run_gc_marks_stale_staged_splits_for_deletion_after_grace_period() { let storage = storage_for_test(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = "test-run-gc--index"; let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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 split_id = "test-run-gc--split"; let split_metadata = SplitMetadata { @@ -354,14 +383,23 @@ mod tests { index_uid: index_uid.clone(), ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 1); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 1 + ); // The staging grace period hasn't passed yet so the split remains staged. run_garbage_collect( @@ -378,7 +416,17 @@ mod tests { let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 1); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 1 + ); // The staging grace period has passed so the split is marked for deletion. run_garbage_collect( @@ -395,18 +443,34 @@ mod tests { let query = ListSplitsQuery::for_index(index_uid).with_split_state(SplitState::MarkedForDeletion); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 1); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 1 + ); } #[tokio::test] async fn test_run_gc_deletes_splits_marked_for_deletion_after_grace_period() { let storage = storage_for_test(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = "test-run-gc--index"; let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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 split_id = "test-run-gc--split"; let split_metadata = SplitMetadata { @@ -414,18 +478,29 @@ mod tests { index_uid: IndexUid::new(index_id), ..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 mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id.to_string()]); metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); - metastore - .mark_splits_for_deletion(index_uid.clone(), &[split_id]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_state(SplitState::MarkedForDeletion); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 1); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 1 + ); // The delete grace period hasn't passed yet so the split remains marked for deletion. run_garbage_collect( @@ -442,7 +517,17 @@ mod tests { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_state(SplitState::MarkedForDeletion); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 1); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 1 + ); // The delete grace period has passed so the split is deleted. run_garbage_collect( @@ -458,22 +543,32 @@ mod tests { .unwrap(); let query = ListSplitsQuery::for_index(index_uid); - assert_eq!(metastore.list_splits(query).await.unwrap().len(), 0); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + assert_eq!( + metastore + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap() + .len(), + 0 + ); } #[tokio::test] async fn test_run_gc_deletes_splits_with_no_split() { // Test that we make only 2 calls to the metastore. let storage = storage_for_test(); - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_list_splits() .times(2) - .returning(|_| Ok(Vec::new())); + .returning(|_| Ok(ListSplitsResponse::empty())); run_garbage_collect( IndexUid::new("index-test-gc-deletes"), storage.clone(), - Arc::new(metastore), + MetastoreServiceClient::from(metastore), Duration::from_secs(30), Duration::from_secs(30), false, @@ -486,12 +581,18 @@ mod tests { #[tokio::test] async fn test_delete_splits_from_storage_and_metastore_happy_path() { let storage = storage_for_test(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = "test-delete-splits-happy--index"; let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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 split_id = "test-delete-splits-happy--split"; let split_metadata = SplitMetadata { @@ -499,12 +600,14 @@ mod tests { index_uid: IndexUid::new(index_id), ..Default::default() }; + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata.clone()) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let mark_splits_for_deletion = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id.to_string()]); metastore - .stage_splits(index_uid.clone(), vec![split_metadata.clone()]) - .await - .unwrap(); - metastore - .mark_splits_for_deletion(index_uid.clone(), &[split_id]) + .mark_splits_for_deletion(mark_splits_for_deletion) .await .unwrap(); @@ -514,7 +617,12 @@ mod tests { storage.put(split_path, payload).await.unwrap(); assert!(storage.exists(split_path).await.unwrap()); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 1); let deleted_split_infos = delete_splits_from_storage_and_metastore( @@ -535,9 +643,11 @@ mod tests { ); assert!(!storage.exists(split_path).await.unwrap()); assert!(metastore - .list_all_splits(index_uid) + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() + .deserialize_splits() + .unwrap() .is_empty()); } @@ -571,12 +681,18 @@ mod tests { Err(bulk_delete_error) }); let storage = Arc::new(mock_storage); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = "test-delete-splits-storage-error--index"; let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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 split_id_0 = "test-delete-splits-storage-error--split-0"; let split_metadata_0 = SplitMetadata { @@ -590,15 +706,18 @@ mod tests { index_uid: index_uid.clone(), ..Default::default() }; + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + [split_metadata_0.clone(), split_metadata_1.clone()], + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( + index_uid.to_string(), + vec![split_id_0.to_string(), split_id_1.to_string()], + ); metastore - .stage_splits( - index_uid.clone(), - vec![split_metadata_0.clone(), split_metadata_1.clone()], - ) - .await - .unwrap(); - metastore - .mark_splits_for_deletion(index_uid.clone(), &[split_id_0, split_id_1]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); @@ -616,7 +735,12 @@ mod tests { assert_eq!(error.storage_failures.len(), 1); assert_eq!(error.metastore_failures.len(), 0); - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 1); assert_eq!(splits[0].split_id(), split_id_1); } @@ -650,13 +774,12 @@ mod tests { let index_id = "test-delete-splits-storage-error--index"; let index_uid = IndexUid::new(index_id.to_string()); - let mut mock_metastore = MockMetastore::new(); - mock_metastore.expect_delete_splits().return_once(|_, _| { + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore.expect_delete_splits().return_once(|_| { Err(MetastoreError::NotFound(EntityKind::Index { index_id: index_id.to_string(), })) }); - let metastore = Arc::new(mock_metastore); let split_id_0 = "test-delete-splits-storage-error--split-0"; let split_metadata_0 = SplitMetadata { @@ -673,7 +796,7 @@ mod tests { let error = delete_splits_from_storage_and_metastore( index_uid.clone(), storage.clone(), - metastore.clone(), + MetastoreServiceClient::from(mock_metastore), vec![split_metadata_0, split_metadata_1], None, ) diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 067b47715b3..62fc88a15de 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -18,17 +18,22 @@ // along with this program. If not, see . use std::path::Path; -use std::sync::Arc; use std::time::Duration; use quickwit_common::fs::{empty_dir, get_cache_directory_path}; use quickwit_config::{validate_identifier, IndexConfig, SourceConfig}; use quickwit_indexing::check_source_connectivity; use quickwit_metastore::{ - IndexMetadata, ListSplitsQuery, Metastore, SplitInfo, SplitMetadata, SplitState, + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitInfo, SplitMetadata, + SplitState, }; -use quickwit_proto::metastore::{EntityKind, MetastoreError}; -use quickwit_proto::{IndexUid, ServiceError, ServiceErrorCode}; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, EntityKind, IndexMetadataRequest, + ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, + MetastoreServiceClient, ResetSourceCheckpointRequest, +}; +use quickwit_proto::{IndexUid, ServiceError, ServiceErrorCode, SplitId}; use quickwit_storage::{StorageResolver, StorageResolverError}; use thiserror::Error; use tracing::{error, info}; @@ -73,26 +78,26 @@ impl ServiceError for IndexServiceError { /// Index service responsible for creating, updating and deleting indexes. #[derive(Clone)] pub struct IndexService { - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, } impl IndexService { /// Creates an `IndexService`. - pub fn new(metastore: Arc, storage_resolver: StorageResolver) -> Self { + pub fn new(metastore: MetastoreServiceClient, storage_resolver: StorageResolver) -> Self { Self { metastore, storage_resolver, } } - pub fn metastore(&self) -> Arc { + pub fn metastore(&self) -> MetastoreServiceClient { self.metastore.clone() } /// Creates an index from `IndexConfig`. pub async fn create_index( - &self, + &mut self, index_config: IndexConfig, overwrite: bool, ) -> Result { @@ -115,19 +120,33 @@ impl IndexService { } } + let mut metastore = self.metastore.clone(); + // Add default ingest-api & cli-ingest sources config. let index_id = index_config.index_id.clone(); - let index_uid = self.metastore.create_index(index_config).await?; - self.metastore - .add_source(index_uid.clone(), SourceConfig::ingest_api_default()) - .await?; - self.metastore - .add_source(index_uid.clone(), SourceConfig::ingest_default()) - .await?; - self.metastore - .add_source(index_uid, SourceConfig::cli_ingest_source()) - .await?; - let index_metadata = self.metastore.index_metadata(&index_id).await?; + let create_index_request = CreateIndexRequest::try_from_index_config(index_config)?; + let create_index_response = metastore.create_index(create_index_request).await?; + let index_uid: IndexUid = create_index_response.index_uid.into(); + let add_ingest_api_source_request = AddSourceRequest::try_from_source_config( + index_uid.clone(), + SourceConfig::ingest_api_default(), + )?; + metastore.add_source(add_ingest_api_source_request).await?; + let add_ingest_source_request = AddSourceRequest::try_from_source_config( + index_uid.clone(), + SourceConfig::ingest_default(), + )?; + metastore.add_source(add_ingest_source_request).await?; + let add_ingest_cli_source_request = AddSourceRequest::try_from_source_config( + index_uid.clone(), + SourceConfig::cli_ingest_source(), + )?; + metastore.add_source(add_ingest_cli_source_request).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; Ok(index_metadata) } @@ -138,20 +157,27 @@ impl IndexService { /// * `index_id` - The target index Id. /// * `dry_run` - Should this only return a list of affected files without performing deletion. pub async fn delete_index( - &self, + &mut self, index_id: &str, dry_run: bool, ) -> Result, IndexServiceError> { - let index_metadata = self.metastore.index_metadata(index_id).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = self + .metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_uri = index_metadata.into_index_config().index_uri.clone(); let storage = self.storage_resolver.resolve(&index_uri).await?; if dry_run { + let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid)?; let splits_to_delete = self .metastore - .list_all_splits(index_uid.clone()) + .list_splits(list_splits_request) .await? + .deserialize_splits()? .into_iter() .map(|split| split.split_metadata.as_split_info()) .collect::>(); @@ -160,25 +186,27 @@ impl IndexService { // Schedule staged and published splits for deletion. let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_states([SplitState::Staged, SplitState::Published]); - let splits = self.metastore.list_splits(query).await?; - let split_ids = splits - .iter() - .map(|split| split.split_id()) - .collect::>(); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let split_ids = self + .metastore + .list_splits(list_splits_request) + .await? + .deserialize_split_ids()?; + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), split_ids); self.metastore - .mark_splits_for_deletion(index_uid.clone(), &split_ids) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await?; // Select splits to delete 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 .metastore - .list_splits(query) + .list_splits(list_splits_request) .await? - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); + .deserialize_splits_metadata()?; let deleted_splits = delete_splits_from_storage_and_metastore( index_uid.clone(), @@ -188,7 +216,10 @@ impl IndexService { None, ) .await?; - self.metastore.delete_index(index_uid).await?; + let delete_index_request = DeleteIndexRequest { + index_uid: index_uid.to_string(), + }; + self.metastore.delete_index(delete_index_request).await?; Ok(deleted_splits) } @@ -199,12 +230,17 @@ impl IndexService { /// * `grace_period` - Threshold period after which a staged split can be garbage collected. /// * `dry_run` - Should this only return a list of affected files without performing deletion. pub async fn garbage_collect_index( - &self, + &mut self, index_id: &str, grace_period: Duration, dry_run: bool, ) -> anyhow::Result { - let index_metadata = self.metastore.index_metadata(index_id).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = self + .metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); let storage = self @@ -237,37 +273,52 @@ impl IndexService { /// * `metastore` - A metastore object for interacting with the metastore. /// * `index_id` - The target index Id. /// * `storage_resolver` - A storage resolver object to access the storage. - pub async fn clear_index(&self, index_id: &str) -> Result<(), IndexServiceError> { - let index_metadata = self.metastore.index_metadata(index_id).await?; + pub async fn clear_index(&mut self, index_id: &str) -> Result<(), IndexServiceError> { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = self + .metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let storage = self .storage_resolver .resolve(index_metadata.index_uri()) .await?; - let splits = self.metastore.list_all_splits(index_uid.clone()).await?; - let split_ids: Vec<&str> = splits.iter().map(|split| split.split_id()).collect(); + let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid.clone())?; + let splits_metadata: Vec = self + .metastore + .list_splits(list_splits_request) + .await? + .deserialize_splits_metadata()?; + let split_ids: Vec = splits_metadata + .iter() + .map(|split| split.split_id.to_string()) + .collect(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), split_ids.clone()); self.metastore - .mark_splits_for_deletion(index_uid.clone(), &split_ids) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await?; - let split_metas: Vec = splits - .into_iter() - .map(|split| split.split_metadata) - .collect(); // FIXME: return an error. if let Err(err) = delete_splits_from_storage_and_metastore( index_uid.clone(), storage, self.metastore.clone(), - split_metas, + splits_metadata, None, ) .await { - error!(metastore_uri=%self.metastore.uri(), index_id=%index_id, error=?err, "failed to delete all the split files during garbage collection"); + error!(metastore_endpoints=?self.metastore.endpoints(), index_id=%index_id, error=?err, "failed to delete all the split files during garbage collection"); } for source_id in index_metadata.sources.keys() { + let reset_source_checkpoint_request = ResetSourceCheckpointRequest { + index_uid: index_uid.to_string(), + source_id: source_id.to_string(), + }; self.metastore - .reset_source_checkpoint(index_uid.clone(), source_id) + .reset_source_checkpoint(reset_source_checkpoint_request) .await?; } Ok(()) @@ -275,7 +326,7 @@ impl IndexService { /// Creates a source config for index `index_id`. pub async fn create_source( - &self, + &mut self, index_uid: IndexUid, source_config: SourceConfig, ) -> Result { @@ -289,18 +340,21 @@ impl IndexService { check_source_connectivity(&self.storage_resolver, &source_config) .await .map_err(IndexServiceError::InvalidConfig)?; - self.metastore - .add_source(index_uid.clone(), source_config) - .await?; + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config.clone())?; + self.metastore.add_source(add_source_request).await?; info!( "source `{}` successfully created for index `{}`", source_id, index_uid.index_id() ); + let index_metadata_request = + IndexMetadataRequest::for_index_id(index_uid.index_id().to_string()); let source = self .metastore - .index_metadata(index_uid.index_id()) + .index_metadata(index_metadata_request) .await? + .deserialize_index_metadata()? .sources .get(&source_id) .ok_or_else(|| { @@ -313,14 +367,16 @@ impl IndexService { } pub async fn get_source( - &self, + &mut self, index_id: &str, source_id: &str, ) -> Result { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let source_config = self .metastore - .index_metadata(index_id) + .index_metadata(index_metadata_request) .await? + .deserialize_index_metadata()? .sources .get(source_id) .ok_or_else(|| { @@ -360,16 +416,17 @@ mod tests { use quickwit_common::uri::Uri; use quickwit_config::IndexConfig; - use quickwit_metastore::{metastore_for_test, SplitMetadata}; + use quickwit_metastore::{metastore_for_test, SplitMetadata, StageSplitsRequestExt}; + use quickwit_proto::metastore::StageSplitsRequest; use quickwit_storage::PutPayload; use super::*; #[tokio::test] async fn test_create_index() { - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let storage_resolver = StorageResolver::for_test(); - let index_service = IndexService::new(metastore.clone(), storage_resolver); + let mut index_service = IndexService::new(metastore.clone(), storage_resolver); let index_id = "test-index"; let index_uri = "ram://indexes/test-index"; let index_config = IndexConfig::for_test(index_id, index_uri); @@ -379,7 +436,10 @@ mod tests { .unwrap(); assert_eq!(index_metadata_0.index_id(), index_id); assert_eq!(index_metadata_0.index_uri(), &index_uri); - assert!(metastore.index_exists(index_id).await.unwrap()); + assert!(metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .is_ok()); let error = index_service .create_index(index_config.clone(), false) @@ -403,13 +463,13 @@ mod tests { #[tokio::test] async fn test_delete_index() { - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let storage_resolver = StorageResolver::for_test(); let storage = storage_resolver .resolve(&Uri::for_test("ram://indexes/test-index")) .await .unwrap(); - let index_service = IndexService::new(metastore.clone(), storage_resolver); + let mut index_service = IndexService::new(metastore.clone(), storage_resolver); let index_id = "test-index"; let index_uri = "ram://indexes/test-index"; let index_config = IndexConfig::for_test(index_id, index_uri); @@ -425,12 +485,19 @@ mod tests { index_uid: index_uid.clone(), ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata.clone()]) + 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(); + + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .deserialize_splits() .unwrap(); - - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); assert_eq!(splits.len(), 1); let split_path_str = format!("{}.split", split_id); @@ -443,7 +510,7 @@ mod tests { assert_eq!(split_infos.len(), 1); let error = metastore - .list_all_splits(index_uid.clone()) + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap_err(); assert!( diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index defacafb39d..d0113ed7971 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -49,8 +49,11 @@ use quickwit_indexing::actors::MergeExecutor; use quickwit_indexing::merge_policy::MergeOperation; use quickwit_indexing::models::MergeScratch; use quickwit_indexing::{get_tantivy_directory_from_split_bundle, TestSandbox}; -use quickwit_metastore::{ListSplitsQuery, Split, SplitMetadata, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitMetadata, SplitState, +}; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; use quickwit_proto::IndexUid; use serde_json::Value as JsonValue; use tantivy::{Directory, Inventory}; @@ -184,7 +187,14 @@ async fn aux_test_failpoints() -> anyhow::Result<()> { test_index_builder.add_documents(batch_2).await?; let query = ListSplitsQuery::for_index(test_index_builder.index_uid()) .with_split_state(SplitState::Published); - let mut splits = test_index_builder.metastore().list_splits(query).await?; + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query).unwrap(); + let mut splits = test_index_builder + .metastore() + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits() + .unwrap(); splits.sort_by_key(|split| *split.split_metadata.time_range.clone().unwrap().start()); assert_eq!(splits.len(), 2); assert_eq!( @@ -256,14 +266,12 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul } tokio::time::sleep(Duration::from_millis(10)).await; - let metastore = test_index_builder.metastore(); - let splits: Vec = metastore - .list_all_splits(test_index_builder.index_uid()) - .await?; - let split_metadatas: Vec = splits - .into_iter() - .map(|split| split.split_metadata) - .collect(); + let mut metastore = test_index_builder.metastore(); + let split_metadatas: Vec = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(test_index_builder.index_uid()).unwrap()) + .await? + .deserialize_splits_metadata() + .unwrap(); let merge_scratch_directory = TempDirectory::for_test(); let downloaded_splits_directory = diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index 05014eb0a3d..97cae5c890a 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -38,8 +38,10 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_config::IndexingSettings; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; -use quickwit_metastore::Metastore; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::{ + LastDeleteOpstampRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::PublishToken; use quickwit_query::get_quickwit_fastfield_normalizer_manager; use serde::Serialize; @@ -81,7 +83,7 @@ pub struct IndexerCounters { struct IndexerState { pipeline_id: IndexingPipelineId, - metastore: Arc, + metastore: MetastoreServiceClient, indexing_directory: TempDirectory, indexing_settings: IndexingSettings, publish_lock: PublishLock, @@ -188,12 +190,17 @@ impl IndexerState { } else { None }; - let last_delete_opstamp = ctx + let last_delete_opstamp_request = LastDeleteOpstampRequest { + index_uid: self.pipeline_id.index_uid.to_string(), + }; + let last_delete_opstamp_response = ctx .protect_future( self.metastore - .last_delete_opstamp(self.pipeline_id.index_uid.clone()), + .clone() + .last_delete_opstamp(last_delete_opstamp_request), ) .await?; + let last_delete_opstamp = last_delete_opstamp_response.last_delete_opstamp; let checkpoint_delta = IndexCheckpointDelta { source_id: self.pipeline_id.source_id.clone(), @@ -486,7 +493,7 @@ impl Indexer { pub fn new( pipeline_id: IndexingPipelineId, doc_mapper: Arc, - metastore: Arc, + metastore: MetastoreServiceClient, indexing_directory: TempDirectory, indexing_settings: IndexingSettings, cooperative_indexing_permits: Option>, @@ -641,7 +648,7 @@ mod tests { use quickwit_actors::Universe; use quickwit_doc_mapper::{default_doc_mapper_for_test, DefaultDocMapper}; use quickwit_metastore::checkpoint::SourceCheckpointDelta; - use quickwit_metastore::MockMetastore; + use quickwit_proto::metastore::{EmptyResponse, LastDeleteOpstampResponse}; use quickwit_proto::IndexUid; use tantivy::{doc, DateTime}; @@ -679,8 +686,9 @@ mod tests { #[tokio::test] async fn test_indexer_triggers_commit_on_target_num_docs() -> anyhow::Result<()> { + let index_uid = IndexUid::new("test-index"); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: index_uid.clone(), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -695,20 +703,20 @@ mod tests { indexing_settings.split_num_docs_target = 3; let universe = Universe::with_accelerated_time(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() .times(2) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(last_delete_opstamp) + .returning(move |delete_opstamp_request| { + assert_eq!(delete_opstamp_request.index_uid, index_uid.to_string()); + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) }); metastore.expect_publish_splits().never(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -814,8 +822,9 @@ mod tests { #[tokio::test] async fn test_indexer_triggers_commit_on_memory_limit() -> anyhow::Result<()> { let universe = Universe::new(); + let index_uid = IndexUid::new("test-index"); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: index_uid.clone(), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -828,20 +837,20 @@ mod tests { let mut indexing_settings = IndexingSettings::for_test(); indexing_settings.resources.heap_size = Byte::from_bytes(5_000_000); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() .times(1..=2) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(last_delete_opstamp) + .returning(move |last_delete_opstamp_request| { + assert_eq!(last_delete_opstamp_request.index_uid, index_uid.to_string()); + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) }); metastore.expect_publish_splits().never(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -906,19 +915,18 @@ mod tests { let mut indexing_settings = IndexingSettings::for_test(); indexing_settings.commit_timeout_secs = 1; let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(last_delete_opstamp) + .returning(move |_last_delete_opstamp_request| { + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) }); metastore.expect_publish_splits().never(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -990,18 +998,17 @@ mod tests { let indexing_directory = TempDirectory::for_test(); let indexing_settings = IndexingSettings::for_test(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(last_delete_opstamp) + .returning(move |_last_delete_opstamp_request| { + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) }); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, Some(Arc::new(Semaphore::new(1))), @@ -1062,20 +1069,17 @@ mod tests { let indexing_directory = TempDirectory::for_test(); let indexing_settings = IndexingSettings::for_test(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() .once() - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1146,20 +1150,17 @@ mod tests { let indexing_directory = TempDirectory::for_test(); let indexing_settings = IndexingSettings::for_test(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore.expect_publish_splits().never(); metastore .expect_last_delete_opstamp() .once() - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1240,20 +1241,17 @@ mod tests { let body_field = doc_mapper.schema().get_field("body").unwrap(); let indexing_directory = TempDirectory::for_test(); let indexing_settings = IndexingSettings::for_test(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_last_delete_opstamp() .times(1) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1314,20 +1312,17 @@ mod tests { let indexing_directory = TempDirectory::for_test(); let mut indexing_settings = IndexingSettings::for_test(); indexing_settings.split_num_docs_target = 1; - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_last_delete_opstamp() .times(2) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1389,20 +1384,17 @@ mod tests { let indexing_directory = TempDirectory::for_test(); let mut indexing_settings = IndexingSettings::for_test(); indexing_settings.split_num_docs_target = 1; - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_last_delete_opstamp() .times(1) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1456,20 +1448,17 @@ mod tests { let body_field = doc_mapper.schema().get_field("body").unwrap(); let indexing_directory = TempDirectory::for_test(); let indexing_settings = IndexingSettings::for_test(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_last_delete_opstamp() .times(1) - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore.expect_publish_splits().never(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), indexing_directory, indexing_settings, None, @@ -1522,23 +1511,22 @@ mod tests { let commit_timeout = indexing_settings.commit_timeout(); let universe = Universe::with_accelerated_time(); let (index_serializer_mailbox, index_serializer_inbox) = universe.create_test_mailbox(); - let mut metastore = MockMetastore::default(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_publish_splits() - .returning(move |_, splits, _, _, _| { - assert!(splits.is_empty()); - Ok(()) - }); - metastore - .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!(index_uid.index_id(), "test-index"); - Ok(last_delete_opstamp) + .returning(move |publish_splits_request| { + assert!(publish_splits_request.replaced_split_ids.is_empty()); + Ok(EmptyResponse {}) }); + mock_metastore.expect_last_delete_opstamp().returning( + move |_last_delete_opstamp_request| { + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) + }, + ); let indexer = Indexer::new( pipeline_id, doc_mapper, - Arc::new(metastore), + MetastoreServiceClient::from(mock_metastore), indexing_directory, indexing_settings, None, diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 730b62e87e1..7b863f08916 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -32,9 +32,11 @@ use quickwit_common::KillSwitch; use quickwit_config::{IndexingSettings, SourceConfig}; use quickwit_doc_mapper::DocMapper; use quickwit_ingest::IngesterPool; -use quickwit_metastore::Metastore; +use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::metastore::MetastoreError; +use quickwit_proto::metastore::{ + IndexMetadataRequest, MetastoreError, MetastoreService, MetastoreServiceClient, +}; use quickwit_storage::{Storage, StorageResolver}; use tokio::sync::Semaphore; use tracing::{debug, error, info, instrument}; @@ -415,9 +417,11 @@ impl IndexingPipeline { .spawn(doc_processor); // Fetch index_metadata to be sure to have the last updated checkpoint. + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_metadata = ctx - .protect_future(self.params.metastore.index_metadata(index_id)) - .await?; + .protect_future(self.params.metastore.index_metadata(index_metadata_request)) + .await? + .deserialize_index_metadata()?; let source_checkpoint = index_metadata .checkpoint .source_checkpoint(source_id) @@ -550,7 +554,7 @@ impl Handler for IndexingPipeline { pub struct IndexingPipelineParams { pub pipeline_id: IndexingPipelineId, - pub metastore: Arc, + pub metastore: MetastoreServiceClient, pub storage: Arc, // Indexing-related parameters @@ -583,8 +587,12 @@ mod tests { use quickwit_actors::{Command, Universe}; use quickwit_config::{IndexingSettings, SourceInputFormat, SourceParams, VoidSourceParams}; use quickwit_doc_mapper::{default_doc_mapper_for_test, DefaultDocMapper}; - use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::metastore::MetastoreError; + use quickwit_metastore::checkpoint::IndexCheckpointDelta; + use quickwit_metastore::{IndexMetadata, ListSplitsResponseExt, PublishSplitsRequestExt}; + use quickwit_proto::metastore::{ + EmptyResponse, IndexMetadataResponse, LastDeleteOpstampResponse, ListSplitsResponse, + MetastoreError, + }; use quickwit_proto::IndexUid; use quickwit_storage::RamStorage; @@ -606,15 +614,19 @@ mod tests { mut num_fails: usize, ) -> anyhow::Result { let universe = Universe::new(); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_index_metadata() - .withf(|index_id| index_id == "test-index") + .withf(|index_metadata_request| { + index_metadata_request.index_id.as_ref().unwrap() == "test-index" + }) .returning(move |_| { if num_fails == 0 { let index_metadata = IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); - return Ok(index_metadata); + return Ok( + IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap(), + ); } num_fails -= 1; Err(MetastoreError::Connection { @@ -623,40 +635,32 @@ mod tests { }); metastore .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!("test-index", index_uid.index_id()); - Ok(10) - }); + .returning(move |_last_delete_opstamp_request| Ok(LastDeleteOpstampResponse::new(10))); metastore .expect_mark_splits_for_deletion() - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); metastore .expect_stage_splits() - .withf(|index_uid, _metadata| -> bool { - *index_uid == "test-index:11111111111111111111111111" + .withf(|stage_splits_request| -> bool { + stage_splits_request.index_uid == "test-index:11111111111111111111111111" }) - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); metastore .expect_publish_splits() - .withf( - |index_uid, - splits, - replaced_splits, - checkpoint_delta_opt, - _publish_token_opt| - -> bool { - let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - *index_uid == "test-index:11111111111111111111111111" - && checkpoint_delta.source_id == "test-source" - && splits.len() == 1 - && replaced_splits.is_empty() - && format!("{:?}", checkpoint_delta.source_delta) - .ends_with(":(00000000000000000000..00000000000000001030])") - }, - ) - .returning(|_, _, _, _, _| Ok(())); + .withf(|publish_splits_request| -> bool { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid == "test-index:11111111111111111111111111" + && checkpoint_delta.source_id == "test-source" + && publish_splits_request.staged_split_ids.len() == 1 + && publish_splits_request.replaced_split_ids.is_empty() + && format!("{:?}", checkpoint_delta.source_delta) + .ends_with(":(00000000000000000000..00000000000000001030])") + }) + .returning(|_| Ok(EmptyResponse {})); let node_id = "test-node"; - let metastore = Arc::new(metastore); let pipeline_id = IndexingPipelineId { index_uid: "test-index:11111111111111111111111111".to_string().into(), source_id: "test-source".to_string(), @@ -684,7 +688,7 @@ mod tests { indexing_directory: TempDirectory::for_test(), indexing_settings: IndexingSettings::for_test(), ingester_pool: IngesterPool::default(), - metastore: metastore.clone(), + metastore: MetastoreServiceClient::from(metastore), storage, split_store, merge_policy: default_merge_policy(), @@ -717,48 +721,46 @@ mod tests { #[tokio::test] async fn test_indexing_pipeline_simple() -> anyhow::Result<()> { - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_index_metadata() - .withf(|index_id| index_id == "test-index") + .withf(|index_metadata_request| { + index_metadata_request.index_id.as_ref().unwrap() == "test-index" + }) .returning(|_| { - Ok(IndexMetadata::for_test( - "test-index", - "ram:///indexes/test-index", - )) + let index_metadata = + IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap()) }); metastore .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!("test-index", index_uid.index_id()); - Ok(10) - }); + .withf(|last_delete_opstamp| { + last_delete_opstamp.index_uid == "test-index:11111111111111111111111111" + }) + .returning(move |_| Ok(LastDeleteOpstampResponse::new(10))); metastore .expect_stage_splits() - .withf(|index_uid, _metadata| *index_uid == "test-index:11111111111111111111111111") - .returning(|_, _| Ok(())); + .withf(|stage_splits_request| { + stage_splits_request.index_uid == "test-index:11111111111111111111111111" + }) + .returning(|_| Ok(EmptyResponse {})); metastore .expect_publish_splits() - .withf( - |index_uid, - splits, - replaced_split_ids, - checkpoint_delta_opt, - _publish_token_opt| - -> bool { - let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - *index_uid == "test-index:11111111111111111111111111" - && splits.len() == 1 - && replaced_split_ids.is_empty() - && checkpoint_delta.source_id == "test-source" - && format!("{:?}", checkpoint_delta.source_delta) - .ends_with(":(00000000000000000000..00000000000000001030])") - }, - ) - .returning(|_, _, _, _, _| Ok(())); + .withf(|publish_splits_request| -> bool { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid == "test-index:11111111111111111111111111" + && publish_splits_request.staged_split_ids.len() == 1 + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_id == "test-source" + && format!("{:?}", checkpoint_delta.source_delta) + .ends_with(":(00000000000000000000..00000000000000001030])") + }) + .returning(|_| Ok(EmptyResponse {})); let universe = Universe::new(); let node_id = "test-node"; - let metastore = Arc::new(metastore); let pipeline_id = IndexingPipelineId { index_uid: "test-index:11111111111111111111111111".to_string().into(), source_id: "test-source".to_string(), @@ -785,7 +787,7 @@ mod tests { indexing_directory: TempDirectory::for_test(), indexing_settings: IndexingSettings::for_test(), ingester_pool: IngesterPool::default(), - metastore: metastore.clone(), + metastore: MetastoreServiceClient::from(metastore), queues_dir_path: PathBuf::from("./queues"), storage, split_store, @@ -809,20 +811,22 @@ mod tests { #[tokio::test] async fn test_merge_pipeline_does_not_stop_on_indexing_pipeline_failure() { - let mut metastore = MockMetastore::default(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_index_metadata() - .withf(|index_id| index_id == "test-index") + .withf(|index_metadata_request| { + index_metadata_request.index_id.as_ref().unwrap() == "test-index" + }) .returning(|_| { - Ok(IndexMetadata::for_test( - "test-index", - "ram:///indexes/test-index", - )) + let index_metadata = + IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap()) }); - metastore.expect_list_splits().returning(|_| Ok(Vec::new())); + mock_metastore + .expect_list_splits() + .returning(|_| Ok(ListSplitsResponse::empty())); let universe = Universe::with_accelerated_time(); let node_id = "test-node"; - let metastore = Arc::new(metastore); let doc_mapper = Arc::new(default_doc_mapper_for_test()); let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new("test-index"), @@ -839,6 +843,7 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; + let metastore = MetastoreServiceClient::from(mock_metastore); let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage.clone()); let merge_pipeline_params = MergePipelineParams { @@ -864,7 +869,7 @@ mod tests { indexing_directory: TempDirectory::for_test(), indexing_settings: IndexingSettings::for_test(), ingester_pool: IngesterPool::default(), - metastore: metastore.clone(), + metastore, queues_dir_path: PathBuf::from("./queues"), storage, split_store, @@ -903,48 +908,44 @@ mod tests { #[tokio::test] async fn test_indexing_pipeline_all_failures_handling() -> anyhow::Result<()> { - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); metastore .expect_index_metadata() - .withf(|index_id| index_id == "test-index") + .withf(|index_metadata_request| { + index_metadata_request.index_id.as_ref().unwrap() == "test-index" + }) .returning(|_| { - Ok(IndexMetadata::for_test( - "test-index", - "ram:///indexes/test-index", - )) + let index_metadata = + IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap()) }); metastore .expect_last_delete_opstamp() - .returning(move |index_uid| { - assert_eq!("test-index", index_uid.index_id()); - Ok(10) - }); + .withf(|last_delete_opstamp| { + last_delete_opstamp.index_uid == "test-index:11111111111111111111111111" + }) + .returning(move |_| Ok(LastDeleteOpstampResponse::new(10))); metastore .expect_stage_splits() .never() - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); metastore .expect_publish_splits() - .withf( - |index_uid, - splits, - replaced_split_ids, - checkpoint_delta_opt, - _publish_token_opt| - -> bool { - let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - *index_uid == "test-index:11111111111111111111111111" - && splits.is_empty() - && replaced_split_ids.is_empty() - && checkpoint_delta.source_id == "test-source" - && format!("{:?}", checkpoint_delta.source_delta) - .ends_with(":(00000000000000000000..00000000000000001030])") - }, - ) - .returning(|_, _, _, _, _| Ok(())); + .withf(|publish_splits_request| -> bool { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid == "test-index:11111111111111111111111111" + && publish_splits_request.staged_split_ids.is_empty() + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_id == "test-source" + && format!("{:?}", checkpoint_delta.source_delta) + .ends_with(":(00000000000000000000..00000000000000001030])") + }) + .returning(|_| Ok(EmptyResponse {})); let universe = Universe::new(); let node_id = "test-node"; - let metastore = Arc::new(metastore); let pipeline_id = IndexingPipelineId { index_uid: "test-index:11111111111111111111111111".to_string().into(), source_id: "test-source".to_string(), @@ -989,7 +990,7 @@ mod tests { indexing_directory: TempDirectory::for_test(), indexing_settings: IndexingSettings::for_test(), ingester_pool: IngesterPool::default(), - metastore: metastore.clone(), + metastore: MetastoreServiceClient::from(metastore), queues_dir_path: PathBuf::from("./queues"), storage, split_store, diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index ead45206170..0860ff76db3 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -40,11 +40,17 @@ use quickwit_config::{ use quickwit_ingest::{ DropQueueRequest, IngestApiService, IngesterPool, ListQueuesRequest, QUEUES_DIR_NAME, }; -use quickwit_metastore::{IndexMetadata, ListIndexesQuery, Metastore}; +use quickwit_metastore::{ + IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataRequestExt, + ListIndexesMetadataResponseExt, +}; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, ApplyIndexingPlanResponse, IndexingError, IndexingPipelineId, IndexingTask, }; +use quickwit_proto::metastore::{ + IndexMetadataRequest, ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::{IndexId, IndexUid}; use quickwit_storage::StorageResolver; use serde::{Deserialize, Serialize}; @@ -102,7 +108,7 @@ pub struct IndexingService { indexing_root_directory: PathBuf, queue_dir_path: PathBuf, cluster: Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, ingest_api_service_opt: Option>, ingester_pool: IngesterPool, storage_resolver: StorageResolver, @@ -135,7 +141,7 @@ impl IndexingService { indexer_config: IndexerConfig, num_blocking_threads: usize, cluster: Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, ingest_api_service_opt: Option>, ingester_pool: IngesterPool, storage_resolver: StorageResolver, @@ -335,11 +341,15 @@ impl IndexingService { index_id: &str, ) -> Result { let _protect_guard = ctx.protect_zone(); - let index_metadata = self + let index_metadata_response = self .metastore - .index_metadata(index_id) + .clone() + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) .await .map_err(|err| IndexingError::MetastoreError(err.to_string()))?; + let index_metadata = index_metadata_response + .deserialize_index_metadata() + .map_err(|err| IndexingError::MetastoreError(err.to_string()))?; Ok(index_metadata) } @@ -648,11 +658,13 @@ impl IndexingService { .collect(); debug!(queues=?queues, "List ingest API queues."); - let index_ids: HashSet = self + let indexes_metadatas = self .metastore - .list_indexes_metadatas(ListIndexesQuery::All) - .await - .context("failed to list queues")? + .clone() + .list_indexes_metadata(ListIndexesMetadataRequest::all()) + .await? + .deserialize_indexes_metadata()?; + let index_ids: HashSet = indexes_metadatas .into_iter() .map(|index_metadata| index_metadata.index_id().to_string()) .collect(); @@ -808,18 +820,26 @@ mod tests { use quickwit_cluster::create_cluster_for_test; use quickwit_common::rand::append_random_suffix; use quickwit_config::{ - IngestApiConfig, SourceConfig, SourceInputFormat, SourceParams, VecSourceParams, + IngestApiConfig, KafkaSourceParams, SourceConfig, SourceInputFormat, SourceParams, + VecSourceParams, }; use quickwit_ingest::{init_ingest_api, CreateQueueIfNotExistsRequest}; - use quickwit_metastore::{metastore_for_test, MockMetastore}; + use quickwit_metastore::{ + metastore_for_test, AddSourceRequestExt, CreateIndexRequestExt, + ListIndexesMetadataResponseExt, ListSplitsResponseExt, + }; use quickwit_proto::indexing::IndexingTask; + use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, IndexMetadataResponse, + ListIndexesMetadataResponse, ListSplitsResponse, + }; use super::*; async fn spawn_indexing_service_for_test( data_dir_path: &Path, universe: &Universe, - metastore: Arc, + metastore: MetastoreServiceClient, cluster: Cluster, ) -> (Mailbox, ActorHandle) { let indexer_config = IndexerConfig::for_test().unwrap(); @@ -854,17 +874,25 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); - metastore - .add_source(index_uid.clone(), SourceConfig::ingest_api_default()) + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) .await - .unwrap(); + .unwrap() + .index_uid + .into(); + let create_source_request = AddSourceRequest::try_from_source_config( + index_uid.clone(), + SourceConfig::ingest_api_default(), + ) + .unwrap(); + metastore.add_source(create_source_request).await.unwrap(); let universe = Universe::with_accelerated_time(); let temp_dir = tempfile::tempdir().unwrap(); let (indexing_service, indexing_service_handle) = @@ -947,13 +975,14 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + metastore.create_index(create_index_request).await.unwrap(); let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); @@ -1001,17 +1030,25 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); - metastore - .add_source(index_uid.clone(), SourceConfig::ingest_api_default()) + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) .await - .unwrap(); + .unwrap() + .index_uid + .into(); + let add_source_request = AddSourceRequest::try_from_source_config( + index_uid.clone(), + SourceConfig::ingest_api_default(), + ) + .unwrap(); + metastore.add_source(add_source_request).await.unwrap(); let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); let (indexing_service, indexing_service_handle) = spawn_indexing_service_for_test( @@ -1032,11 +1069,16 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - metastore - .add_source(index_uid.clone(), source_config_1.clone()) + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config_1.clone()) + .unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + let metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.clone())) .await + .unwrap() + .deserialize_index_metadata() .unwrap(); - let metadata = metastore.index_metadata(index_id.as_str()).await.unwrap(); let indexing_tasks = vec![ IndexingTask { index_uid: metadata.index_uid.to_string(), @@ -1061,19 +1103,25 @@ mod tests { 2 ); + let kafka_params = KafkaSourceParams { + topic: "my-topic".to_string(), + client_log_level: None, + client_params: serde_json::Value::Null, + enable_backfill_mode: false, + }; let source_config_2 = SourceConfig { source_id: "test-indexing-service--source-2".to_string(), max_num_pipelines_per_indexer: NonZeroUsize::new(2).unwrap(), desired_num_pipelines: NonZeroUsize::new(2).unwrap(), enabled: true, - source_params: SourceParams::void(), + source_params: SourceParams::Kafka(kafka_params), transform_config: None, input_format: SourceInputFormat::Json, }; - metastore - .add_source(index_uid.clone(), source_config_2.clone()) - .await - .unwrap(); + let add_source_request_2 = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config_2.clone()) + .unwrap(); + metastore.add_source(add_source_request_2).await.unwrap(); let indexing_tasks = vec![ IndexingTask { @@ -1179,7 +1227,12 @@ mod tests { ); // Delete index and apply empty plan - metastore.delete_index(index_uid).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); indexing_service .ask_for_res(ApplyIndexingPlanRequest { indexing_tasks: Vec::new(), @@ -1201,7 +1254,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -1216,11 +1269,17 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - let index_uid = metastore.create_index(index_config).await.unwrap(); - metastore - .add_source(index_uid.clone(), source_config.clone()) + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) .await - .unwrap(); + .unwrap() + .index_uid + .into(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config.clone()) + .unwrap(); + metastore.add_source(add_source_request).await.unwrap(); // Test `IndexingService::new`. let temp_dir = tempfile::tempdir().unwrap(); @@ -1341,21 +1400,30 @@ mod tests { index_metadata .sources .insert(source_config.source_id.clone(), source_config.clone()); - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata_clone = index_metadata.clone(); - metastore.expect_list_indexes_metadatas().returning( - move |_list_indexes_query: ListIndexesQuery| Ok(vec![index_metadata_clone.clone()]), - ); metastore - .expect_index_metadata() - .returning(move |_| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(|_| Ok(Vec::new())); + .expect_list_indexes_metadata() + .returning(move |_request| { + let list_indexes_metadatas_response = + ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata_clone.clone(), + ]) + .unwrap(); + Ok(list_indexes_metadatas_response) + }); + metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + metastore + .expect_list_splits() + .returning(|_| Ok(ListSplitsResponse::try_from_splits(Vec::new()).unwrap())); let universe = Universe::new(); let temp_dir = tempfile::tempdir().unwrap(); let (indexing_service, indexing_service_handle) = spawn_indexing_service_for_test( temp_dir.path(), &universe, - Arc::new(metastore), + MetastoreServiceClient::from(metastore), cluster, ) .await; @@ -1398,8 +1466,14 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let metastore = metastore_for_test(); - let index_uid = metastore.create_index(index_config).await.unwrap(); + let mut metastore = metastore_for_test(); + 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(); // Setup ingest api objects let universe = Universe::with_accelerated_time(); @@ -1440,7 +1514,12 @@ mod tests { indexing_server.run_ingest_api_queues_gc().await.unwrap(); assert_eq!(indexing_server.counters.num_deleted_queues, 0); - metastore.delete_index(index_uid).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); indexing_server.run_ingest_api_queues_gc().await.unwrap(); assert_eq!(indexing_server.counters.num_deleted_queues, 1); diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index aea557efd7b..7376465d17e 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -33,9 +33,12 @@ use quickwit_common::runtimes::RuntimeType; use quickwit_common::temp_dir::TempDirectory; use quickwit_directories::UnionDirectory; use quickwit_doc_mapper::DocMapper; -use quickwit_metastore::{Metastore, SplitMetadata}; +use quickwit_metastore::SplitMetadata; use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::metastore::DeleteTask; +use quickwit_proto::metastore::{ + DeleteTask, ListDeleteTasksRequest, MarkSplitsForDeletionRequest, MetastoreService, + MetastoreServiceClient, +}; use quickwit_query::get_quickwit_fastfield_normalizer_manager; use quickwit_query::query_ast::QueryAst; use tantivy::directory::{DirectoryClone, MmapDirectory, RamDirectory}; @@ -54,7 +57,7 @@ use crate::models::{IndexedSplit, IndexedSplitBatch, MergeScratch, PublishLock, #[derive(Clone)] pub struct MergeExecutor { pipeline_id: IndexingPipelineId, - metastore: Arc, + metastore: MetastoreServiceClient, doc_mapper: Arc, io_controls: IoControls, merge_packager_mailbox: Mailbox, @@ -271,7 +274,7 @@ fn max_merge_ops(splits: &[SplitMetadata]) -> usize { impl MergeExecutor { pub fn new( pipeline_id: IndexingPipelineId, - metastore: Arc, + metastore: MetastoreServiceClient, doc_mapper: Arc, io_controls: IoControls, merge_packager_mailbox: Mailbox, @@ -334,12 +337,12 @@ impl MergeExecutor { merge_scratch_directory: TempDirectory, ctx: &ActorContext, ) -> anyhow::Result> { + let list_delete_tasks_request = + ListDeleteTasksRequest::new(split.index_uid.to_string(), split.delete_opstamp); let delete_tasks = ctx - .protect_future( - self.metastore - .list_delete_tasks(split.index_uid.clone(), split.delete_opstamp), - ) - .await?; + .protect_future(self.metastore.list_delete_tasks(list_delete_tasks_request)) + .await? + .delete_tasks; if delete_tasks.is_empty() { warn!( "No delete task found for split `{}` with `delete_optamp` = `{}`.", @@ -389,8 +392,12 @@ impl MergeExecutor { "All documents from split `{}` were deleted.", split.split_id() ); + let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( + split.index_uid.to_string(), + vec![split.split_id.to_string()], + ); self.metastore - .mark_splits_for_deletion(split.index_uid.clone(), &[split.split_id()]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await?; return Ok(None); }; @@ -533,8 +540,12 @@ fn open_index>>( mod tests { use quickwit_actors::Universe; use quickwit_common::split_file; - use quickwit_metastore::SplitMetadata; - use quickwit_proto::metastore::DeleteQuery; + use quickwit_metastore::{ + ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, StageSplitsRequestExt, + }; + use quickwit_proto::metastore::{ + DeleteQuery, ListSplitsRequest, PublishSplitsRequest, StageSplitsRequest, + }; use serde_json::Value as JsonValue; use tantivy::{Document, Inventory, ReloadPolicy, TantivyDocument}; @@ -570,13 +581,14 @@ mod tests { ); test_sandbox.add_documents(single_doc).await?; } - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); + let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid).unwrap(); let split_metas: Vec = metastore - .list_all_splits(index_uid) - .await? - .into_iter() - .map(|split| split.split_metadata) - .collect(); + .list_splits(list_splits_request) + .await + .unwrap() + .deserialize_splits_metadata() + .unwrap(); assert_eq!(split_metas.len(), 4); let merge_scratch_directory = TempDirectory::for_test(); let downloaded_splits_directory = @@ -689,7 +701,7 @@ mod tests { source_id: "unknown".to_string(), }; test_sandbox.add_documents(docs).await?; - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); metastore .create_delete_task(DeleteQuery { index_uid: index_uid.to_string(), @@ -698,29 +710,34 @@ mod tests { query_ast: quickwit_query::query_ast::qast_json_helper(delete_query, &["body"]), }) .await?; - let split_metadata = metastore - .list_all_splits(index_uid.clone()) - .await? + let split = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap() .into_iter() .next() - .unwrap() - .split_metadata; + .unwrap(); // We want to test a delete on a split with num_merge_ops > 0. - let mut new_split_metadata = split_metadata.clone(); + let mut new_split_metadata = split.split_metadata.clone(); new_split_metadata.split_id = new_split_id(); new_split_metadata.num_merge_ops = 1; + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + new_split_metadata.clone(), + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); + 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()], + index_checkpoint_delta_json_opt: None, + publish_token_opt: None, + }; metastore - .stage_splits(index_uid.clone(), vec![new_split_metadata.clone()]) - .await - .unwrap(); - metastore - .publish_splits( - index_uid.clone(), - &[new_split_metadata.split_id()], - &[split_metadata.split_id()], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); let expected_uncompressed_docs_size_in_bytes = @@ -728,7 +745,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_metadata.split_id()); + let split_filename = split_file(split.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 @@ -805,10 +822,13 @@ mod tests { } } else { assert!(packager_msgs.is_empty()); - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); assert!(metastore - .list_all_splits(index_uid) - .await? + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap() .into_iter() .all( |split| split.split_state == quickwit_metastore::SplitState::MarkedForDeletion diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 9254d4a9b3a..0294f4cd8a2 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -31,9 +31,13 @@ use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_common::KillSwitch; use quickwit_doc_mapper::DocMapper; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, +}; use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::metastore::MetastoreError; +use quickwit_proto::metastore::{ + ListSplitsRequest, MetastoreError, MetastoreService, MetastoreServiceClient, +}; use time::OffsetDateTime; use tracing::{debug, error, info, instrument}; @@ -218,13 +222,16 @@ impl MergePipeline { let query = ListSplitsQuery::for_index(self.params.pipeline_id.index_uid.clone()) .with_split_state(SplitState::Published) .retain_immature(OffsetDateTime::now_utc()); - let published_splits = ctx - .protect_future(self.params.metastore.list_splits(query)) + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let published_splits_metadata: Vec = ctx + .protect_future(self.params.metastore.list_splits(list_splits_request)) .await? - .into_iter() - .map(|split| split.split_metadata) - .collect::>(); + .deserialize_splits_metadata()?; + info!( + "splits_metadata.len() = {}", + published_splits_metadata.len() + ); // Merge publisher let merge_publisher = Publisher::new( PublisherType::MergePublisher, @@ -314,7 +321,7 @@ impl MergePipeline { // Merge planner let merge_planner = MergePlanner::new( self.params.pipeline_id.clone(), - published_splits, + published_splits_metadata, self.params.merge_policy.clone(), merge_split_downloader_mailbox, ); @@ -462,7 +469,7 @@ pub struct MergePipelineParams { pub pipeline_id: IndexingPipelineId, pub doc_mapper: Arc, pub indexing_directory: TempDirectory, - pub metastore: Arc, + pub metastore: MetastoreServiceClient, pub split_store: IndexingSplitStore, pub merge_policy: Arc, pub max_concurrent_split_uploads: usize, //< TODO share with the indexing pipeline. @@ -478,8 +485,9 @@ mod tests { use quickwit_actors::{ActorExitStatus, Universe}; use quickwit_common::temp_dir::TempDirectory; use quickwit_doc_mapper::default_doc_mapper_for_test; - use quickwit_metastore::MockMetastore; + use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; use quickwit_proto::indexing::IndexingPipelineId; + use quickwit_proto::metastore::{ListSplitsResponse, MetastoreServiceClient}; use quickwit_proto::IndexUid; use quickwit_storage::RamStorage; @@ -489,7 +497,7 @@ mod tests { #[tokio::test] async fn test_merge_pipeline_simple() -> anyhow::Result<()> { - let mut metastore = MockMetastore::default(); + let mut metastore = MetastoreServiceClient::mock(); let index_uid = IndexUid::new("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), @@ -500,7 +508,8 @@ mod tests { metastore .expect_list_splits() .times(1) - .returning(move |list_split_query| { + .withf(move |list_splits_request| { + let list_split_query = list_splits_request.deserialize_list_splits_query().unwrap(); assert_eq!(list_split_query.index_uids, &[index_uid.clone()]); assert_eq!( list_split_query.split_states, @@ -509,8 +518,9 @@ mod tests { let Bound::Excluded(_) = list_split_query.mature else { panic!("Expected excluded bound."); }; - Ok(Vec::new()) - }); + true + }) + .returning(|_| Ok(ListSplitsResponse::try_from_splits(Vec::new()).unwrap())); let universe = Universe::with_accelerated_time(); let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage.clone()); @@ -518,7 +528,7 @@ mod tests { pipeline_id, doc_mapper: Arc::new(default_doc_mapper_for_test()), indexing_directory: TempDirectory::for_test(), - metastore: Arc::new(metastore), + metastore: MetastoreServiceClient::from(metastore), split_store, merge_policy: default_merge_policy(), max_concurrent_split_uploads: 2, diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 72ba3a65a7e..d0d2586ade8 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -17,13 +17,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; - use anyhow::Context; use async_trait::async_trait; use fail::fail_point; use quickwit_actors::{Actor, ActorContext, Handler, Mailbox, QueueCapacity}; -use quickwit_metastore::Metastore; +use quickwit_proto::metastore::{MetastoreService, MetastoreServiceClient, PublishSplitsRequest}; use serde::Serialize; use tracing::{info, instrument}; @@ -56,7 +54,7 @@ impl PublisherType { #[derive(Clone)] pub struct Publisher { publisher_type: PublisherType, - metastore: Arc, + metastore: MetastoreServiceClient, merge_planner_mailbox_opt: Option>, source_mailbox_opt: Option>, counters: PublisherCounters, @@ -65,7 +63,7 @@ pub struct Publisher { impl Publisher { pub fn new( publisher_type: PublisherType, - metastore: Arc, + metastore: MetastoreServiceClient, merge_planner_mailbox_opt: Option>, source_mailbox_opt: Option>, ) -> Publisher { @@ -121,21 +119,26 @@ impl Handler for Publisher { .. } = split_update; - let split_ids: Vec<&str> = new_splits.iter().map(|split| split.split_id()).collect(); - - let replaced_split_ids_ref_vec: Vec<&str> = - replaced_split_ids.iter().map(String::as_str).collect(); - + let index_checkpoint_delta_json_opt = checkpoint_delta_opt + .as_ref() + .map(serde_json::to_string) + .transpose() + .context("failed to serialize `IndexCheckpointDelta`")?; + let split_ids: Vec = new_splits + .iter() + .map(|split| split.split_id.clone()) + .collect(); if let Some(_guard) = publish_lock.acquire().await { - ctx.protect_future(self.metastore.publish_splits( - index_uid, - &split_ids[..], - &replaced_split_ids_ref_vec, - checkpoint_delta_opt.clone(), - publish_token_opt, - )) - .await - .context("failed to publish splits")?; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + staged_split_ids: split_ids.clone(), + replaced_split_ids: replaced_split_ids.clone(), + index_checkpoint_delta_json_opt, + publish_token_opt: publish_token_opt.clone(), + }; + ctx.protect_future(self.metastore.publish_splits(publish_splits_request)) + .await + .context("failed to publish splits")?; } else { // TODO: Remove the junk right away? info!( @@ -192,7 +195,8 @@ mod tests { use quickwit_metastore::checkpoint::{ IndexCheckpointDelta, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, }; - use quickwit_metastore::{MockMetastore, SplitMetadata}; + use quickwit_metastore::{PublishSplitsRequestExt, SplitMetadata}; + use quickwit_proto::metastore::EmptyResponse; use quickwit_proto::IndexUid; use tracing::Span; @@ -202,32 +206,29 @@ mod tests { #[tokio::test] async fn test_publisher_publish_operation() { let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_publish_splits() - .withf( - |index_uid, - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - _publish_token_opt| { - let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - *index_uid == "index:11111111111111111111111111" - && checkpoint_delta.source_id == "source" - && split_ids[..] == ["split"] - && replaced_split_ids.is_empty() - && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) - }, - ) + .withf(|publish_splits_request| { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid == "index:11111111111111111111111111" + && checkpoint_delta.source_id == "source" + && publish_splits_request.staged_split_ids[..] == ["split"] + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) + }) .times(1) - .returning(|_, _, _, _, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); let (source_mailbox, source_inbox) = universe.create_test_mailbox(); let publisher = Publisher::new( PublisherType::MainPublisher, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), Some(merge_planner_mailbox), Some(source_mailbox), ); @@ -279,32 +280,29 @@ mod tests { #[tokio::test] async fn test_publisher_publish_operation_with_empty_splits() { let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_publish_splits() - .withf( - |index_uid, - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - _publish_token_opt| { - let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - *index_uid == "index:11111111111111111111111111" - && checkpoint_delta.source_id == "source" - && split_ids.is_empty() - && replaced_split_ids.is_empty() - && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) - }, - ) + .withf(|publish_splits_request| { + let checkpoint_delta: IndexCheckpointDelta = publish_splits_request + .deserialize_index_checkpoint() + .unwrap() + .unwrap(); + publish_splits_request.index_uid == "index:11111111111111111111111111" + && checkpoint_delta.source_id == "source" + && publish_splits_request.staged_split_ids.is_empty() + && publish_splits_request.replaced_split_ids.is_empty() + && checkpoint_delta.source_delta == SourceCheckpointDelta::from_range(1..3) + }) .times(1) - .returning(|_, _, _, _, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); let (source_mailbox, source_inbox) = universe.create_test_mailbox(); let publisher = Publisher::new( PublisherType::MainPublisher, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), Some(merge_planner_mailbox), Some(source_mailbox), ); @@ -354,27 +352,23 @@ mod tests { #[tokio::test] async fn test_publisher_replace_operation() { let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_publish_splits() - .withf( - |index_uid, - new_split_ids, - replaced_split_ids, - checkpoint_delta_opt, - _publish_token_opt| { - *index_uid == "index:11111111111111111111111111" - && new_split_ids[..] == ["split3"] - && replaced_split_ids[..] == ["split1", "split2"] - && checkpoint_delta_opt.is_none() - }, - ) + .withf(|publish_splits_requests| { + publish_splits_requests.index_uid == "index:11111111111111111111111111" + && publish_splits_requests.staged_split_ids[..] == ["split3"] + && publish_splits_requests.replaced_split_ids[..] == ["split1", "split2"] + && publish_splits_requests + .index_checkpoint_delta_json_opt() + .is_empty() + }) .times(1) - .returning(|_, _, _, _, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); let publisher = Publisher::new( PublisherType::MainPublisher, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), Some(merge_planner_mailbox), None, ); @@ -408,13 +402,13 @@ mod tests { #[tokio::test] async fn publisher_acquires_publish_lock() { let universe = Universe::with_accelerated_time(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore.expect_publish_splits().never(); let (merge_planner_mailbox, merge_planner_inbox) = universe.create_test_mailbox(); let publisher = Publisher::new( PublisherType::MainPublisher, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), Some(merge_planner_mailbox), None, ); diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index dff910ea6cd..be116722b0c 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -31,7 +31,8 @@ use once_cell::sync::OnceCell; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, QueueCapacity}; use quickwit_common::pubsub::EventBroker; use quickwit_metastore::checkpoint::IndexCheckpointDelta; -use quickwit_metastore::{Metastore, SplitMetadata}; +use quickwit_metastore::{SplitMetadata, StageSplitsRequestExt}; +use quickwit_proto::metastore::{MetastoreService, MetastoreServiceClient, StageSplitsRequest}; use quickwit_proto::search::{ReportSplit, ReportSplitsRequest}; use quickwit_proto::{IndexUid, PublishToken}; use quickwit_storage::SplitPayloadBuilder; @@ -162,7 +163,7 @@ impl SplitsUpdateSender { #[derive(Clone)] pub struct Uploader { uploader_type: UploaderType, - metastore: Arc, + metastore: MetastoreServiceClient, merge_policy: Arc, split_store: IndexingSplitStore, split_update_mailbox: SplitsUpdateMailbox, @@ -174,7 +175,7 @@ pub struct Uploader { impl Uploader { pub fn new( uploader_type: UploaderType, - metastore: Arc, + metastore: MetastoreServiceClient, merge_policy: Arc, split_store: IndexingSplitStore, split_update_mailbox: SplitsUpdateMailbox, @@ -335,8 +336,10 @@ impl Handler for Uploader { } + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata(index_uid.clone(), split_metadata_list.clone())?; metastore - .stage_splits(index_uid.clone(), split_metadata_list.clone()) + .clone() + .stage_splits(stage_splits_request) .await?; counters.num_staged_splits.fetch_add(split_metadata_list.len() as u64, Ordering::SeqCst); @@ -485,8 +488,8 @@ mod tests { use quickwit_common::pubsub::EventSubscriber; use quickwit_common::temp_dir::TempDirectory; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_metastore::MockMetastore; use quickwit_proto::indexing::IndexingPipelineId; + use quickwit_proto::metastore::EmptyResponse; use quickwit_storage::RamStorage; use tantivy::DateTime; use tokio::sync::oneshot; @@ -508,24 +511,26 @@ mod tests { }; let (sequencer_mailbox, sequencer_inbox) = universe.create_test_mailbox::>(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_stage_splits() - .withf(move |index_uid, metadata| -> bool { - let metadata = &metadata[0]; + .withf(move |stage_splits_request| -> bool { + let splits_metadata = stage_splits_request.deserialize_splits_metadata().unwrap(); + let split_metadata = &splits_metadata[0]; + let index_uid: IndexUid = stage_splits_request.index_uid.clone().into(); index_uid.index_id() == "test-index" - && metadata.split_id() == "test-split" - && metadata.time_range == Some(1628203589..=1628203640) + && split_metadata.split_id() == "test-split" + && split_metadata.time_range == Some(1628203589..=1628203640) }) .times(1) - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let ram_storage = RamStorage::default(); let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new(ram_storage.clone())); let merge_policy = Arc::new(NopMergePolicy); let uploader = Uploader::new( UploaderType::IndexUploader, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), merge_policy, split_store, SplitsUpdateMailbox::Sequencer(sequencer_mailbox), @@ -618,26 +623,27 @@ mod tests { let universe = Universe::new(); let (sequencer_mailbox, sequencer_inbox) = universe.create_test_mailbox::>(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_stage_splits() - .withf(move |index_uid, metadata_list| -> bool { - let is_metadata_valid = metadata_list.iter().all(|metadata| { + .withf(move |stage_splits_request| -> bool { + let splits_metadata = stage_splits_request.deserialize_splits_metadata().unwrap(); + let is_metadata_valid = splits_metadata.iter().all(|metadata| { ["test-split-1", "test-split-2"].contains(&metadata.split_id()) && metadata.time_range == Some(1628203589..=1628203640) }); - + let index_uid: IndexUid = stage_splits_request.index_uid.clone().into(); index_uid.index_id() == "test-index" && is_metadata_valid }) .times(1) - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let ram_storage = RamStorage::default(); let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new(ram_storage.clone())); let merge_policy = Arc::new(NopMergePolicy); let uploader = Uploader::new( UploaderType::IndexUploader, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), merge_policy, split_store, SplitsUpdateMailbox::Sequencer(sequencer_mailbox), @@ -756,28 +762,29 @@ mod tests { #[tokio::test] async fn test_uploader_without_sequencer() -> anyhow::Result<()> { let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index-no-sequencer"), + index_uid: IndexUid::from("test-index-no-sequencer:11111111111111111111111111"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, }; let universe = Universe::new(); let (publisher_mailbox, publisher_inbox) = universe.create_test_mailbox::(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_stage_splits() - .withf(move |index_uid, _| -> bool { - index_uid.index_id() == "test-index-no-sequencer" + .withf(move |stage_splits_request| -> bool { + stage_splits_request.index_uid + == "test-index-no-sequencer:11111111111111111111111111" }) .times(1) - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let ram_storage = RamStorage::default(); let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new(ram_storage.clone())); let merge_policy = Arc::new(NopMergePolicy); let uploader = Uploader::new( UploaderType::IndexUploader, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), merge_policy, split_store, SplitsUpdateMailbox::Publisher(publisher_mailbox), @@ -839,14 +846,14 @@ mod tests { let universe = Universe::new(); let (sequencer_mailbox, sequencer_inbox) = universe.create_test_mailbox::>(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore.expect_stage_splits().never(); let ram_storage = RamStorage::default(); let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new(ram_storage.clone())); let uploader = Uploader::new( UploaderType::IndexUploader, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), default_merge_policy(), split_store, SplitsUpdateMailbox::Sequencer(sequencer_mailbox), @@ -941,11 +948,11 @@ mod tests { node_id: "test-node".to_string(), pipeline_ord: 0, }; - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_stage_splits() .times(1) - .returning(|_, _| Ok(())); + .returning(|_| Ok(EmptyResponse {})); let ram_storage = RamStorage::default(); let split_store = IndexingSplitStore::create_without_local_store_for_test(Arc::new(ram_storage.clone())); @@ -953,7 +960,7 @@ mod tests { let (publisher_mailbox, _publisher_inbox) = universe.create_test_mailbox(); let uploader = Uploader::new( UploaderType::IndexUploader, - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), merge_policy, split_store, SplitsUpdateMailbox::Publisher(publisher_mailbox), diff --git a/quickwit/quickwit-indexing/src/lib.rs b/quickwit/quickwit-indexing/src/lib.rs index 482f992fb1b..3fd3932eea5 100644 --- a/quickwit/quickwit-indexing/src/lib.rs +++ b/quickwit/quickwit-indexing/src/lib.rs @@ -19,14 +19,12 @@ #![deny(clippy::disallowed_methods)] -use std::sync::Arc; - use quickwit_actors::{Mailbox, Universe}; use quickwit_cluster::Cluster; use quickwit_common::pubsub::EventBroker; use quickwit_config::NodeConfig; use quickwit_ingest::{IngestApiService, IngesterPool}; -use quickwit_metastore::Metastore; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_storage::StorageResolver; use tracing::info; @@ -69,7 +67,7 @@ pub async fn start_indexing_service( config: &NodeConfig, num_blocking_threads: usize, cluster: Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, ingest_api_service: Mailbox, ingester_pool: IngesterPool, storage_resolver: StorageResolver, diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index b333f75b916..a068b7b0c11 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -28,11 +28,12 @@ use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_ingest::{decoded_mrecords, IngesterPool, MRecord, MultiFetchStream}; use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; -use quickwit_metastore::Metastore; use quickwit_proto::ingest::ingester::{ FetchResponseV2, IngesterService, TruncateRequest, TruncateSubrequest, }; -use quickwit_proto::metastore::{AcquireShardsRequest, AcquireShardsSubrequest}; +use quickwit_proto::metastore::{ + AcquireShardsRequest, AcquireShardsSubrequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::types::NodeId; use quickwit_proto::{IndexUid, PublishToken, ShardId, SourceId}; use serde_json::json; @@ -104,7 +105,7 @@ struct AssignedShard { /// Streams documents from a set of shards. pub struct IngestSource { client_id: ClientId, - metastore: Arc, + metastore: MetastoreServiceClient, ingester_pool: IngesterPool, assigned_shards: HashMap, fetch_stream: MultiFetchStream, @@ -440,7 +441,6 @@ mod tests { use quickwit_actors::{ActorContext, Universe}; use quickwit_common::ServiceStream; use quickwit_config::{SourceConfig, SourceParams}; - use quickwit_metastore::MockMetastore; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::ingest::ingester::{IngesterServiceClient, TruncateResponse}; use quickwit_proto::ingest::{MRecordBatch, Shard}; @@ -461,7 +461,7 @@ mod tests { pipeline_ord: 0, }; let source_config = SourceConfig::for_test("test-source", SourceParams::Ingest); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_acquire_shards() .once() @@ -501,8 +501,6 @@ mod tests { }; Ok(response) }); - let metastore = Arc::new(mock_metastore); - let ingester_pool = IngesterPool::default(); let mut ingester_mock_0 = IngesterServiceClient::mock(); @@ -545,7 +543,7 @@ mod tests { let runtime_args = Arc::new(SourceRuntimeArgs { pipeline_id, source_config, - metastore, + metastore: MetastoreServiceClient::from(mock_metastore), ingester_pool: ingester_pool.clone(), queues_dir_path: PathBuf::from("./queues"), storage_resolver: StorageResolver::for_test(), @@ -617,14 +615,13 @@ mod tests { pipeline_ord: 0, }; let source_config = SourceConfig::for_test("test-source", SourceParams::Ingest); - let mock_metastore = MockMetastore::default(); - let metastore = Arc::new(mock_metastore); + let mock_metastore = MetastoreServiceClient::mock(); let ingester_pool = IngesterPool::default(); let runtime_args = Arc::new(SourceRuntimeArgs { pipeline_id, source_config, - metastore, + metastore: MetastoreServiceClient::from(mock_metastore), ingester_pool: ingester_pool.clone(), queues_dir_path: PathBuf::from("./queues"), storage_resolver: StorageResolver::for_test(), @@ -730,8 +727,7 @@ mod tests { pipeline_ord: 0, }; let source_config = SourceConfig::for_test("test-source", SourceParams::Ingest); - let mock_metastore = MockMetastore::default(); - let metastore = Arc::new(mock_metastore); + let mock_metastore = MetastoreServiceClient::mock(); let ingester_pool = IngesterPool::default(); @@ -801,7 +797,7 @@ mod tests { let runtime_args = Arc::new(SourceRuntimeArgs { pipeline_id, source_config, - metastore, + metastore: MetastoreServiceClient::from(mock_metastore), ingester_pool: ingester_pool.clone(), queues_dir_path: PathBuf::from("./queues"), storage_resolver: StorageResolver::for_test(), diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index 58f4f1254d2..0b30531f009 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -30,6 +30,8 @@ use oneshot; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_config::KafkaSourceParams; use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService}; use quickwit_proto::IndexUid; use rdkafka::config::{ClientConfig, RDKafkaLogLevel}; use rdkafka::consumer::{ @@ -346,19 +348,17 @@ impl KafkaSource { partitions: &[i32], assignment_tx: oneshot::Sender>, ) -> anyhow::Result<()> { + let index_metadata_request = + IndexMetadataRequest::for_index_uid(self.ctx.index_uid().to_string()); let index_metadata = ctx .protect_future( self.ctx .metastore - .index_metadata_strict(self.ctx.index_uid()), + .clone() + .index_metadata(index_metadata_request), ) - .await - .with_context(|| { - format!( - "failed to fetch index metadata for index `{}`", - self.ctx.index_id() - ) - })?; + .await? + .deserialize_index_metadata()?; let checkpoint = index_metadata .checkpoint .source_checkpoint(self.ctx.source_id()) @@ -785,7 +785,13 @@ mod kafka_broker_tests { use quickwit_common::rand::append_random_suffix; use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_metastore::{metastore_for_test, Metastore, SplitMetadata}; + use quickwit_metastore::{ + metastore_for_test, CreateIndexRequestExt, SplitMetadata, StageSplitsRequestExt, + }; + use quickwit_proto::metastore::{ + CreateIndexRequest, MetastoreService, MetastoreServiceClient, PublishSplitsRequest, + StageSplitsRequest, + }; use quickwit_proto::IndexUid; use rdkafka::admin::{AdminClient, AdminOptions, NewTopic, TopicReplication}; use rdkafka::client::DefaultClientContext; @@ -926,24 +932,29 @@ mod kafka_broker_tests { } async fn setup_index( - metastore: Arc, + mut metastore: MetastoreServiceClient, index_id: &str, source_id: &str, partition_deltas: &[(u64, i64, i64)], ) -> IndexUid { let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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(); if partition_deltas.is_empty() { return index_uid; } let split_id = new_split_id(); let split_metadata = SplitMetadata::for_test(split_id.clone()); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); let mut source_delta = SourceCheckpointDelta::default(); for (partition_id, from_position, to_position) in partition_deltas { @@ -965,14 +976,16 @@ mod kafka_broker_tests { source_id: source_id.to_string(), source_delta, }; + let checkpoint_delta_json = serde_json::to_string(&checkpoint_delta).unwrap(); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + index_checkpoint_delta_json_opt: Some(checkpoint_delta_json), + staged_split_ids: vec![split_id.clone()], + replaced_split_ids: Vec::new(), + publish_token_opt: None, + }; metastore - .publish_splits( - index_uid.clone(), - &[&split_id], - &[], - Some(checkpoint_delta), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); index_uid diff --git a/quickwit/quickwit-indexing/src/source/mod.rs b/quickwit/quickwit-indexing/src/source/mod.rs index 9129193ff7c..1ef016ecc95 100644 --- a/quickwit/quickwit-indexing/src/source/mod.rs +++ b/quickwit/quickwit-indexing/src/source/mod.rs @@ -73,7 +73,6 @@ mod vec_source; mod void_source; use std::path::PathBuf; -use std::sync::Arc; use std::time::Duration; #[cfg(not(any(feature = "kafka", feature = "kinesis", feature = "pulsar")))] @@ -95,8 +94,8 @@ use quickwit_common::runtimes::RuntimeType; use quickwit_config::{SourceConfig, SourceParams}; use quickwit_ingest::IngesterPool; use quickwit_metastore::checkpoint::{SourceCheckpoint, SourceCheckpointDelta}; -use quickwit_metastore::Metastore; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::{IndexUid, ShardId}; use quickwit_storage::StorageResolver; use serde_json::Value as JsonValue; @@ -116,7 +115,7 @@ use crate::source::ingest_api_source::IngestApiSourceFactory; pub struct SourceRuntimeArgs { pub pipeline_id: IndexingPipelineId, pub source_config: SourceConfig, - pub metastore: Arc, + pub metastore: MetastoreServiceClient, pub ingester_pool: IngesterPool, // Ingest API queues directory path. pub queues_dir_path: PathBuf, @@ -148,9 +147,10 @@ impl SourceRuntimeArgs { fn for_test( index_uid: IndexUid, source_config: SourceConfig, - metastore: Arc, + metastore: MetastoreServiceClient, queues_dir_path: PathBuf, - ) -> Arc { + ) -> std::sync::Arc { + use std::sync::Arc; let pipeline_id = IndexingPipelineId { node_id: "test-node".to_string(), index_uid, diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index 3ee5deb2d93..9f418eaeaa4 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -433,7 +433,6 @@ mod pulsar_broker_tests { use std::num::NonZeroUsize; use std::ops::Range; use std::path::PathBuf; - use std::sync::Arc; use futures::future::join_all; use quickwit_actors::{ActorHandle, Inbox, Universe, HEARTBEAT}; @@ -442,7 +441,13 @@ mod pulsar_broker_tests { use quickwit_metastore::checkpoint::{ IndexCheckpointDelta, PartitionId, Position, SourceCheckpointDelta, }; - use quickwit_metastore::{metastore_for_test, Metastore, SplitMetadata}; + use quickwit_metastore::{ + metastore_for_test, CreateIndexRequestExt, SplitMetadata, StageSplitsRequestExt, + }; + use quickwit_proto::metastore::{ + CreateIndexRequest, MetastoreService, MetastoreServiceClient, PublishSplitsRequest, + StageSplitsRequest, + }; use reqwest::StatusCode; use super::*; @@ -479,24 +484,29 @@ mod pulsar_broker_tests { } async fn setup_index( - metastore: Arc, + mut metastore: MetastoreServiceClient, index_id: &str, source_id: &str, partition_deltas: &[(&str, Position, Position)], ) -> IndexUid { let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(index_id, &index_uri); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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(); if partition_deltas.is_empty() { return index_uid; } let split_id = new_split_id(); let split_metadata = SplitMetadata::for_test(split_id.clone()); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); let mut source_delta = SourceCheckpointDelta::default(); for (partition_id, from_position, to_position) in partition_deltas { @@ -512,14 +522,17 @@ mod pulsar_broker_tests { source_id: source_id.to_string(), source_delta, }; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + staged_split_ids: vec![split_id.clone()], + replaced_split_ids: Vec::new(), + index_checkpoint_delta_json_opt: Some( + serde_json::to_string(&checkpoint_delta).unwrap(), + ), + publish_token_opt: None, + }; metastore - .publish_splits( - index_uid.clone(), - &[&split_id], - &[], - Some(checkpoint_delta), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); index_uid @@ -682,7 +695,7 @@ mod pulsar_broker_tests { async fn create_source( universe: &Universe, - metastore: Arc, + metastore: MetastoreServiceClient, index_uid: IndexUid, source_config: SourceConfig, start_checkpoint: SourceCheckpoint, diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 77ffd30429f..fe6d36ddcbd 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -34,7 +34,10 @@ use quickwit_config::{ }; use quickwit_doc_mapper::DocMapper; use quickwit_ingest::{init_ingest_api, IngesterPool, QUEUES_DIR_NAME}; -use quickwit_metastore::{Metastore, MetastoreResolver, Split, SplitMetadata, SplitState}; +use quickwit_metastore::{ + CreateIndexRequestExt, MetastoreResolver, Split, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::{CreateIndexRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::IndexUid; use quickwit_storage::{Storage, StorageResolver}; use serde_json::Value as JsonValue; @@ -51,7 +54,7 @@ pub struct TestSandbox { index_uid: IndexUid, indexing_service: Mailbox, doc_mapper: Arc, - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, storage: Arc, add_docs_id: AtomicUsize, @@ -95,10 +98,15 @@ impl TestSandbox { let storage_resolver = StorageResolver::for_test(); let metastore_resolver = MetastoreResolver::configured(storage_resolver.clone(), &MetastoreConfigs::default()); - let metastore = metastore_resolver + let mut metastore = metastore_resolver .resolve(&Uri::from_well_formed(METASTORE_URI)) .await?; - let index_uid = metastore.create_index(index_config.clone()).await?; + let create_index_request = CreateIndexRequest::try_from_index_config(index_config.clone())?; + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await? + .index_uid + .into(); let storage = storage_resolver.resolve(&index_uri).await?; let universe = Universe::with_accelerated_time(); let queues_dir_path = temp_dir.path().join(QUEUES_DIR_NAME); @@ -182,7 +190,7 @@ impl TestSandbox { /// The metastore is a file-backed metastore. /// Its data can be found via the `storage` in /// the `ram://quickwit-test-indexes` directory. - pub fn metastore(&self) -> Arc { + pub fn metastore(&self) -> MetastoreServiceClient { self.metastore.clone() } @@ -270,6 +278,9 @@ pub fn mock_split_meta(split_id: &str, index_uid: &IndexUid) -> SplitMetadata { #[cfg(test)] mod tests { + use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; + use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; + use super::TestSandbox; #[tokio::test] @@ -291,16 +302,26 @@ mod tests { serde_json::json!({"title": "Ganimede", "body": "...", "url": "http://ganimede"}), ]).await?; assert_eq!(statistics.num_uploaded_splits, 1); - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); { - let splits = metastore.list_all_splits(test_sandbox.index_uid()).await?; + let splits = metastore + .list_splits( + ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap(), + ) + .await? + .deserialize_splits()?; assert_eq!(splits.len(), 1); test_sandbox.add_documents(vec![ serde_json::json!({"title": "Byzantine-Ottoman wars", "body": "...", "url": "http://biz-ottoman"}), ]).await?; } { - let splits = metastore.list_all_splits(test_sandbox.index_uid()).await?; + let splits = metastore + .list_splits( + ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap(), + ) + .await? + .deserialize_splits()?; assert_eq!(splits.len(), 2); } test_sandbox.assert_quit().await; diff --git a/quickwit/quickwit-ingest/build.rs b/quickwit/quickwit-ingest/build.rs index fff28e6b4f2..fa18ecafc0d 100644 --- a/quickwit/quickwit-ingest/build.rs +++ b/quickwit/quickwit-ingest/build.rs @@ -30,6 +30,7 @@ fn main() { "crate::Result", "crate::IngestServiceError", false, + false, &[], prost_config, ) diff --git a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs index ba67726d065..2d85fa58ec2 100644 --- a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs +++ b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs @@ -197,6 +197,12 @@ impl IngestServiceClient { where T: IngestService, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < MockIngestService + > (), + "`MockIngestService` must be wrapped in a `MockIngestServiceWrapper`. Use `MockIngestService::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( diff --git a/quickwit/quickwit-jaeger/src/integration_tests.rs b/quickwit/quickwit-jaeger/src/integration_tests.rs index f81c0d8bafd..d9253991b98 100644 --- a/quickwit/quickwit-jaeger/src/integration_tests.rs +++ b/quickwit/quickwit-jaeger/src/integration_tests.rs @@ -33,13 +33,16 @@ use quickwit_ingest::{ init_ingest_api, CommitType, CreateQueueRequest, IngestApiService, IngestServiceClient, IngesterPool, QUEUES_DIR_NAME, }; -use quickwit_metastore::{FileBackedMetastore, Metastore}; +use quickwit_metastore::{AddSourceRequestExt, CreateIndexRequestExt, FileBackedMetastore}; use quickwit_opentelemetry::otlp::OtlpGrpcTracesService; use quickwit_proto::jaeger::storage::v1::span_reader_plugin_server::SpanReaderPlugin; use quickwit_proto::jaeger::storage::v1::{ FindTraceIDsRequest, GetOperationsRequest, GetServicesRequest, GetTraceRequest, Operation, SpansResponseChunk, TraceQueryParameters, }; +use quickwit_proto::metastore::{ + AddSourceRequest, CreateIndexRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_server::TraceService; use quickwit_proto::opentelemetry::proto::collector::trace::v1::ExportTraceServiceRequest; use quickwit_proto::opentelemetry::proto::common::v1::any_value::Value as OtlpAnyValueValue; @@ -51,6 +54,7 @@ use quickwit_proto::opentelemetry::proto::trace::v1::span::{Event as OtlpEvent, use quickwit_proto::opentelemetry::proto::trace::v1::{ ResourceSpans, ScopeSpans, Span as OtlpSpan, Status as OtlpStatus, }; +use quickwit_proto::IndexUid; use quickwit_search::{ start_searcher_service, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, @@ -326,19 +330,19 @@ async fn ingester_for_test( (ingester_service, ingester_client) } -async fn metastore_for_test(storage_resolver: &StorageResolver) -> Arc { +async fn metastore_for_test(storage_resolver: &StorageResolver) -> MetastoreServiceClient { let storage = storage_resolver .resolve(&Uri::for_test("ram:///metastore")) .await .unwrap(); - Arc::new(FileBackedMetastore::for_test(storage)) + MetastoreServiceClient::new(FileBackedMetastore::for_test(storage)) } async fn indexer_for_test( universe: &Universe, data_dir_path: &Path, cluster: Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, ingester_service: Mailbox, ingester_pool: IngesterPool, @@ -363,7 +367,7 @@ async fn indexer_for_test( async fn searcher_for_test( cluster: &Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, ) -> Arc { let searcher_config = SearcherConfig::default(); @@ -392,7 +396,7 @@ async fn searcher_for_test( async fn setup_traces_index( temp_dir: &TempDir, - metastore: Arc, + mut metastore: MetastoreServiceClient, ingester_service: &Mailbox, indexer_service: &Mailbox, ) { @@ -401,12 +405,17 @@ async fn setup_traces_index( .unwrap(); let index_config = OtlpGrpcTracesService::index_config(&index_root_uri).unwrap(); let index_id = index_config.index_id.clone(); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - let source_config = SourceConfig::ingest_api_default(); - metastore - .add_source(index_uid.clone(), source_config.clone()) + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) .await - .unwrap(); + .unwrap() + .index_uid + .into(); + let source_config = SourceConfig::ingest_api_default(); + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), source_config.clone()).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); let create_queue_request = CreateQueueRequest { queue_id: index_id.clone(), diff --git a/quickwit/quickwit-janitor/Cargo.toml b/quickwit/quickwit-janitor/Cargo.toml index 435820583ef..de5d7240743 100644 --- a/quickwit/quickwit-janitor/Cargo.toml +++ b/quickwit/quickwit-janitor/Cargo.toml @@ -53,5 +53,6 @@ quickwit-common = { workspace = true, features = ["testsuite"] } quickwit-config = { workspace = true, features = ["testsuite"] } quickwit-indexing = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } quickwit-search = { workspace = true, features = ["testsuite"] } quickwit-storage = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index bd61a79f828..19e5adc5d71 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -36,8 +36,9 @@ use quickwit_indexing::actors::{ }; use quickwit_indexing::merge_policy::merge_policy_from_settings; use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; -use quickwit_metastore::Metastore; +use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::Storage; @@ -78,7 +79,7 @@ pub struct DeleteTaskPipelineState { pub struct DeleteTaskPipeline { index_uid: IndexUid, - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, index_storage: Arc, delete_service_task_dir: PathBuf, @@ -128,7 +129,7 @@ impl Actor for DeleteTaskPipeline { impl DeleteTaskPipeline { pub fn new( index_uid: IndexUid, - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, index_storage: Arc, delete_service_task_dir: PathBuf, @@ -154,11 +155,14 @@ impl DeleteTaskPipeline { root_dir=%self.delete_service_task_dir.to_str().unwrap(), "Spawning delete tasks pipeline.", ); - let index_metadata = self + let index_config = self .metastore - .index_metadata_strict(&self.index_uid) - .await?; - let index_config = index_metadata.into_index_config(); + .index_metadata(IndexMetadataRequest::for_index_uid( + self.index_uid.to_string(), + )) + .await? + .deserialize_index_metadata()? + .into_index_config(); let publisher = Publisher::new( PublisherType::MergePublisher, self.metastore.clone(), @@ -289,8 +293,8 @@ mod tests { use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::TempDirectory; use quickwit_indexing::TestSandbox; - use quickwit_metastore::SplitState; - use quickwit_proto::metastore::DeleteQuery; + use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt, SplitState}; + use quickwit_proto::metastore::{DeleteQuery, ListSplitsRequest, MetastoreService}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{ searcher_pool_for_test, MockSearchService, SearchError, SearchJobPlacer, @@ -349,7 +353,7 @@ mod tests { serde_json::json!({"body": "delete", "ts": 0 }), ]; test_sandbox.add_documents(docs).await?; - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); metastore .create_delete_task(DeleteQuery { index_uid: index_uid.to_string(), @@ -414,7 +418,12 @@ mod tests { assert_eq!(pipeline_state.publisher.metrics.num_errors, 0); let _ = pipeline_mailbox.ask(GracefulShutdown).await; - let splits = metastore.list_all_splits(index_uid).await?; + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(splits.len(), 2); let published_split = splits .iter() diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index 96fa8e52597..53c31cae5e9 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -18,7 +18,6 @@ // along with this program. If not, see . use std::collections::{HashMap, HashSet}; -use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; @@ -29,8 +28,11 @@ use quickwit_common::uri::Uri; use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; use quickwit_indexing::actors::MergeSplitDownloader; use quickwit_indexing::merge_policy::MergeOperation; -use quickwit_metastore::{split_tag_filter, split_time_range_filter, Metastore, Split}; -use quickwit_proto::metastore::{DeleteTask, MetastoreResult}; +use quickwit_metastore::{split_tag_filter, split_time_range_filter, ListSplitsResponseExt, Split}; +use quickwit_proto::metastore::{ + DeleteTask, LastDeleteOpstampRequest, ListDeleteTasksRequest, ListStaleSplitsRequest, + MetastoreResult, MetastoreService, MetastoreServiceClient, UpdateSplitsDeleteOpstampRequest, +}; use quickwit_proto::search::SearchRequest; use quickwit_proto::IndexUid; use quickwit_search::{jobs_to_leaf_requests, IndexMetasForLeafSearch, SearchJob, SearchJobPlacer}; @@ -76,7 +78,7 @@ pub struct DeleteTaskPlanner { index_uid: IndexUid, index_uri: Uri, doc_mapper_str: String, - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, merge_split_downloader_mailbox: Mailbox, /// Inventory of ongoing delete operations. If everything goes well, @@ -120,7 +122,7 @@ impl DeleteTaskPlanner { index_uid: IndexUid, index_uri: Uri, doc_mapper_str: String, - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, merge_split_downloader_mailbox: Mailbox, ) -> Self { @@ -139,10 +141,14 @@ impl DeleteTaskPlanner { async fn send_delete_operations(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { // Loop until there is no more stale splits. loop { + let last_delete_opstamp_request = LastDeleteOpstampRequest { + index_uid: self.index_uid.to_string(), + }; let last_delete_opstamp = self .metastore - .last_delete_opstamp(self.index_uid.clone()) - .await?; + .last_delete_opstamp(last_delete_opstamp_request) + .await? + .last_delete_opstamp; let stale_splits = self .get_relevant_stale_splits(self.index_uid.clone(), last_delete_opstamp, ctx) .await?; @@ -170,13 +176,17 @@ impl DeleteTaskPlanner { // Updates `delete_opstamp` of splits that won't undergo delete operations. let split_ids_without_delete = splits_without_deletes .iter() - .map(|split| split.split_id()) + .map(|split| split.split_id().to_string()) .collect_vec(); - ctx.protect_future(self.metastore.update_splits_delete_opstamp( - self.index_uid.clone(), - &split_ids_without_delete, - last_delete_opstamp, - )) + let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { + index_uid: self.index_uid.to_string(), + split_ids: split_ids_without_delete.clone(), + delete_opstamp: last_delete_opstamp, + }; + ctx.protect_future( + self.metastore + .update_splits_delete_opstamp(update_splits_delete_opstamp_request), + ) .await?; // Sends delete operations. @@ -206,7 +216,7 @@ impl DeleteTaskPlanner { /// Identifies splits that contain documents to delete and /// splits that do not and returns the two groups. async fn partition_splits_by_deletes( - &self, + &mut self, stale_splits: &[Split], ctx: &ActorContext, ) -> anyhow::Result<(Vec, Vec)> { @@ -214,12 +224,14 @@ impl DeleteTaskPlanner { let mut splits_with_deletes: Vec = Vec::new(); for stale_split in stale_splits { + let list_delete_tasks_request = ListDeleteTasksRequest::new( + self.index_uid.to_string(), + stale_split.split_metadata.delete_opstamp, + ); let pending_tasks = ctx - .protect_future(self.metastore.list_delete_tasks( - self.index_uid.clone(), - stale_split.split_metadata.delete_opstamp, - )) - .await?; + .protect_future(self.metastore.list_delete_tasks(list_delete_tasks_request)) + .await? + .delete_tasks; // Keep only delete tasks that matches the split metadata. let pending_and_matching_metadata_tasks = pending_tasks @@ -237,8 +249,8 @@ impl DeleteTaskPlanner { let delete_query_ast = serde_json::from_str(&delete_query.query_ast) .expect("Failed to deserialize query_ast json"); let tags_filter = extract_tags_from_query(delete_query_ast); - split_time_range_filter(stale_split, time_range.as_ref()) - && split_tag_filter(stale_split, tags_filter.as_ref()) + split_time_range_filter(&stale_split.split_metadata, time_range.as_ref()) + && split_tag_filter(&stale_split.split_metadata, tags_filter.as_ref()) }) .collect_vec(); @@ -327,18 +339,20 @@ impl DeleteTaskPlanner { /// Fetches stale splits from [`Metastore`] and excludes immature splits and split already among /// ongoing delete operations. async fn get_relevant_stale_splits( - &self, + &mut self, index_uid: IndexUid, last_delete_opstamp: u64, ctx: &ActorContext, ) -> MetastoreResult> { + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: last_delete_opstamp, + num_splits: NUM_STALE_SPLITS_TO_FETCH as u64, + }; let stale_splits = ctx - .protect_future(self.metastore.list_stale_splits( - index_uid.clone(), - last_delete_opstamp, - NUM_STALE_SPLITS_TO_FETCH, - )) - .await?; + .protect_future(self.metastore.list_stale_splits(list_stale_splits_request)) + .await? + .deserialize_splits()?; debug!( index_id = index_uid.index_id(), last_delete_opstamp = last_delete_opstamp, @@ -408,8 +422,8 @@ mod tests { use quickwit_config::build_doc_mapper; use quickwit_indexing::merge_policy::MergeOperation; use quickwit_indexing::TestSandbox; - use quickwit_metastore::SplitMetadata; - use quickwit_proto::metastore::DeleteQuery; + use quickwit_metastore::{IndexMetadataResponseExt, ListSplitsRequestExt, SplitMetadata}; + use quickwit_proto::metastore::{DeleteQuery, IndexMetadataRequest, ListSplitsRequest}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{searcher_pool_for_test, MockSearchService}; use tantivy::TrackedObject; @@ -448,16 +462,22 @@ mod tests { for doc in docs { test_sandbox.add_documents(vec![doc]).await?; } - let metastore = test_sandbox.metastore(); - let index_metadata = metastore.index_metadata(index_id).await?; + let mut metastore = test_sandbox.metastore(); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); let split_metas: Vec = metastore - .list_all_splits(index_uid.clone()) - .await? - .into_iter() - .map(|split| split.split_metadata) - .collect_vec(); + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap() + .deserialize_splits_metadata() + .unwrap(); assert_eq!(split_metas.len(), 3); let doc_mapper = build_doc_mapper(&index_config.doc_mapping, &index_config.search_settings)?; @@ -568,7 +588,12 @@ mod tests { ); // The other splits has just their delete opstamps updated to the last opstamps which is 2 // as there are 2 delete tasks. The last split - let all_splits = metastore.list_all_splits(index_uid).await?; + let all_splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!(all_splits[0].split_metadata.delete_opstamp, 2); assert_eq!(all_splits[1].split_metadata.delete_opstamp, 2); // The last split has not yet its delete opstamp updated. diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs index 2273b33b3b9..5e91dd50851 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs @@ -19,7 +19,6 @@ use std::collections::{HashMap, HashSet}; use std::path::PathBuf; -use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; @@ -27,7 +26,12 @@ use quickwit_actors::{Actor, ActorContext, ActorExitStatus, ActorHandle, Handler use quickwit_common::pubsub::EventBroker; use quickwit_common::temp_dir::{self}; use quickwit_config::IndexConfig; -use quickwit_metastore::{ListIndexesQuery, Metastore}; +use quickwit_metastore::{ + IndexMetadataResponseExt, ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, +}; +use quickwit_proto::metastore::{ + IndexMetadataRequest, ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::IndexUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::StorageResolver; @@ -52,7 +56,7 @@ pub struct DeleteTaskServiceState { } pub struct DeleteTaskService { - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, storage_resolver: StorageResolver, delete_service_task_dir: PathBuf, @@ -63,7 +67,7 @@ pub struct DeleteTaskService { impl DeleteTaskService { pub async fn new( - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, storage_resolver: StorageResolver, data_dir_path: PathBuf, @@ -112,8 +116,9 @@ impl DeleteTaskService { ) -> anyhow::Result<()> { let mut index_config_by_index_id: HashMap = self .metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await? + .deserialize_indexes_metadata()? .into_iter() .map(|index_metadata| { ( @@ -163,10 +168,13 @@ impl DeleteTaskService { ) -> anyhow::Result<()> { let index_uri = index_config.index_uri.clone(); let index_storage = self.storage_resolver.resolve(&index_uri).await?; + let index_metadata_request = + IndexMetadataRequest::for_index_id(index_config.index_id.to_string()); let index_metadata = self .metastore - .index_metadata(index_config.index_id.as_str()) - .await?; + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let pipeline = DeleteTaskPipeline::new( index_metadata.index_uid.clone(), self.metastore.clone(), @@ -209,7 +217,9 @@ impl Handler for DeleteTaskService { mod tests { use quickwit_common::pubsub::EventBroker; use quickwit_indexing::TestSandbox; - use quickwit_proto::metastore::DeleteQuery; + use quickwit_proto::metastore::{ + DeleteIndexRequest, DeleteQuery, ListDeleteTasksRequest, MetastoreService, + }; use quickwit_search::{searcher_pool_for_test, MockSearchService, SearchJobPlacer}; use quickwit_storage::StorageResolver; @@ -229,7 +239,7 @@ mod tests { "#; let test_sandbox = TestSandbox::create(index_id, doc_mapping_yaml, "{}", &["body"]).await?; let index_uid = test_sandbox.index_uid(); - let metastore = test_sandbox.metastore(); + let mut metastore = test_sandbox.metastore(); let mock_search_service = MockSearchService::new(); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1000", mock_search_service)]); let search_job_placer = SearchJobPlacer::new(searcher_pool); @@ -263,13 +273,19 @@ mod tests { // Just test creation of delete query. assert_eq!( metastore - .list_delete_tasks(index_uid.clone(), 0) + .list_delete_tasks(ListDeleteTasksRequest::new(index_uid.to_string(), 0)) .await .unwrap() + .delete_tasks .len(), 1 ); - metastore.delete_index(index_uid.clone()).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); test_sandbox .universe() .sleep(UPDATE_PIPELINES_INTERVAL * 2) diff --git a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs index 39bfcfb1816..957bb7a6665 100644 --- a/quickwit/quickwit-janitor/src/actors/garbage_collector.rs +++ b/quickwit/quickwit-janitor/src/actors/garbage_collector.rs @@ -19,7 +19,6 @@ use std::collections::HashSet; use std::path::Path; -use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; @@ -29,7 +28,10 @@ use quickwit_actors::{Actor, ActorContext, Handler}; // use quickwit_index_management::run_garbage_collect; use quickwit_common::shared_consts::DELETION_GRACE_PERIOD; use quickwit_index_management::run_garbage_collect; -use quickwit_metastore::{ListIndexesQuery, Metastore}; +use quickwit_metastore::{ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt}; +use quickwit_proto::metastore::{ + ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_storage::StorageResolver; use serde::Serialize; use tracing::{error, info}; @@ -66,13 +68,13 @@ struct Loop; /// An actor for collecting garbage periodically from an index. pub struct GarbageCollector { - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, counters: GarbageCollectorCounters, } impl GarbageCollector { - pub fn new(metastore: Arc, storage_resolver: StorageResolver) -> Self { + pub fn new(metastore: MetastoreServiceClient, storage_resolver: StorageResolver) -> Self { Self { metastore, storage_resolver, @@ -88,9 +90,11 @@ impl GarbageCollector { let indexes = match self .metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await - { + .and_then(|list_indexes_metadata_response| { + list_indexes_metadata_response.deserialize_indexes_metadata() + }) { Ok(metadatas) => metadatas, Err(error) => { error!(error=?error, "Failed to list indexes from the metastore."); @@ -205,13 +209,18 @@ impl Handler for GarbageCollector { mod tests { use std::ops::Bound; use std::path::Path; + use std::sync::Arc; use quickwit_actors::Universe; use quickwit_common::shared_consts::DELETION_GRACE_PERIOD; use quickwit_metastore::{ - IndexMetadata, ListSplitsQuery, MockMetastore, Split, SplitMetadata, SplitState, + IndexMetadata, ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitMetadata, + SplitState, }; - use quickwit_proto::metastore::MetastoreError; + use quickwit_proto::metastore::{ + EmptyResponse, ListIndexesMetadataResponse, ListSplitsResponse, MetastoreError, + }; + use quickwit_proto::IndexUid; use quickwit_storage::MockStorage; use time::OffsetDateTime; @@ -252,11 +261,12 @@ mod tests { Ok(()) }); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_list_splits() .times(2) - .returning(|query: ListSplitsQuery| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert_eq!( query.index_uids[0].to_string(), "test-index:11111111111111111111111111" @@ -283,38 +293,43 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .times(1) - .returning(|index_uid, split_ids| { + .returning(|mark_splits_for_deletion_request| { assert_eq!( - index_uid.to_string(), + mark_splits_for_deletion_request.index_uid, "test-index:11111111111111111111111111" ); - assert_eq!(split_ids, vec!["a"]); - Ok(()) + assert_eq!(mark_splits_for_deletion_request.split_ids, vec!["a"]); + Ok(EmptyResponse {}) }); mock_metastore .expect_delete_splits() .times(1) - .returning(|index_uid, split_ids| { + .returning(|delete_splits_request| { assert_eq!( - index_uid.to_string(), + delete_splits_request.index_uid, "test-index:11111111111111111111111111" ); - let split_ids = HashSet::<&str>::from_iter(split_ids.iter().copied()); + let split_ids = HashSet::<&str>::from_iter( + delete_splits_request + .split_ids + .iter() + .map(|split_id| split_id.as_str()), + ); let expected_split_ids = HashSet::<&str>::from_iter(["a", "b", "c"]); assert_eq!(split_ids, expected_split_ids); - Ok(()) + Ok(EmptyResponse {}) }); let result = run_garbage_collect( "test-index:11111111111111111111111111".to_string().into(), Arc::new(mock_storage), - Arc::new(mock_metastore), + MetastoreServiceClient::from(mock_metastore), STAGED_GRACE_PERIOD, DELETION_GRACE_PERIOD, false, @@ -327,20 +342,25 @@ mod tests { #[tokio::test] async fn test_garbage_collect_calls_dependencies_appropriately() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) - .returning(move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![IndexMetadata::for_test( + .returning(|_list_indexes_request| { + let indexes_metadata = vec![IndexMetadata::for_test( "test-index", "ram://indexes/test-index", - )]) + )]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore .expect_list_splits() .times(2) - .returning(|query| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert_eq!(query.index_uids[0].index_id(), "test-index"); let splits = match query.split_states[0] { SplitState::Staged => make_splits(&["a"], SplitState::Staged), @@ -349,31 +369,40 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .times(1) - .returning(|index_uid, split_ids| { + .returning(|mark_splits_for_deletion_request| { + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.clone().into(); assert_eq!(index_uid.index_id(), "test-index"); - assert_eq!(split_ids, vec!["a"]); - Ok(()) + assert_eq!(mark_splits_for_deletion_request.split_ids, vec!["a"]); + Ok(EmptyResponse {}) }); mock_metastore .expect_delete_splits() .times(1) - .returning(|index_uid, split_ids| { + .returning(|delete_splits_request| { + let index_uid: IndexUid = delete_splits_request.index_uid.clone().into(); assert_eq!(index_uid.index_id(), "test-index"); - let split_ids = HashSet::<&str>::from_iter(split_ids.iter().copied()); + let split_ids = HashSet::<&str>::from_iter( + delete_splits_request + .split_ids + .iter() + .map(|split_id| split_id.as_str()), + ); let expected_split_ids = HashSet::<&str>::from_iter(["a", "b", "c"]); assert_eq!(split_ids, expected_split_ids); - Ok(()) + Ok(EmptyResponse {}) }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handler) = universe.spawn_builder().spawn(garbage_collect_actor); @@ -388,20 +417,25 @@ mod tests { #[tokio::test] async fn test_garbage_collect_get_calls_repeatedly() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(3) - .returning(move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![IndexMetadata::for_test( + .returning(|_list_indexes_metadata| { + let indexes_metadata = vec![IndexMetadata::for_test( "test-index", "ram://indexes/test-index", - )]) + )]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore .expect_list_splits() .times(6) - .returning(|query| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert_eq!(query.index_uids[0].index_id(), "test-index"); let splits = match query.split_states[0] { SplitState::Staged => make_splits(&["a"], SplitState::Staged), @@ -410,31 +444,40 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .times(3) - .returning(|index_uid, split_ids| { + .returning(|mark_splits_for_deletion_request| { + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.clone().into(); assert_eq!(index_uid.index_id(), "test-index"); - assert_eq!(split_ids, vec!["a"]); - Ok(()) + assert_eq!(mark_splits_for_deletion_request.split_ids, vec!["a"]); + Ok(EmptyResponse {}) }); mock_metastore .expect_delete_splits() .times(3) - .returning(|index_uid, split_ids| { + .returning(|delete_splits_request| { + let index_uid: IndexUid = delete_splits_request.index_uid.clone().into(); assert_eq!(index_uid.index_id(), "test-index"); - let split_ids = HashSet::<&str>::from_iter(split_ids.iter().copied()); + let split_ids = HashSet::<&str>::from_iter( + delete_splits_request + .split_ids + .iter() + .map(|split_id| split_id.as_str()), + ); let expected_split_ids = HashSet::<&str>::from_iter(["a", "b"]); assert_eq!(split_ids, expected_split_ids); - Ok(()) + Ok(EmptyResponse {}) }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(garbage_collect_actor); @@ -474,18 +517,20 @@ mod tests { #[tokio::test] async fn test_garbage_collect_get_called_repeatedly_on_failure() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(4) - .returning(move |_list_indexes_query: ListIndexesQuery| { + .returning(|_list_indexes_request| { Err(MetastoreError::Db { message: "fail to list indexes".to_string(), }) }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(garbage_collect_actor); @@ -505,19 +550,25 @@ mod tests { #[tokio::test] async fn test_garbage_collect_fails_to_resolve_storage() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) - .returning(move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![IndexMetadata::for_test( + .returning(move |_list_indexes_request| { + let indexes_metadata = vec![IndexMetadata::for_test( "test-index", "postgresql://indexes/test-index", - )]) + )]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(garbage_collect_actor); @@ -535,20 +586,25 @@ mod tests { #[tokio::test] async fn test_garbage_collect_fails_to_run_gc_on_one_index() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) - .returning(move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![ + .returning(|_list_indexes_request| { + let indexes_metadata = vec![ IndexMetadata::for_test("test-index-1", "ram:///indexes/test-index-1"), IndexMetadata::for_test("test-index-2", "ram:///indexes/test-index-2"), - ]) + ]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore .expect_list_splits() .times(3) - .returning(|query| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert!(["test-index-1", "test-index-2"].contains(&query.index_uids[0].index_id())); if query.index_uids[0].index_id() == "test-index-2" { @@ -563,29 +619,37 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .once() - .returning(|index_uid, split_ids| { + .returning(|mark_splits_for_deletion_request| { + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.clone().into(); assert!(["test-index-1", "test-index-2"].contains(&index_uid.index_id())); - assert_eq!(split_ids, vec!["a"]); - Ok(()) + assert_eq!(mark_splits_for_deletion_request.split_ids, vec!["a"]); + Ok(EmptyResponse {}) }); mock_metastore .expect_delete_splits() .once() - .returning(|_index_id, split_ids| { - let split_ids = HashSet::<&str>::from_iter(split_ids.iter().copied()); + .returning(|delete_splits_request| { + let split_ids = HashSet::<&str>::from_iter( + delete_splits_request + .split_ids + .iter() + .map(|split_id| split_id.as_str()), + ); let expected_split_ids = HashSet::<&str>::from_iter(["a", "b"]); assert_eq!(split_ids, expected_split_ids); - Ok(()) + Ok(EmptyResponse {}) }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(garbage_collect_actor); @@ -603,20 +667,25 @@ mod tests { #[tokio::test] async fn test_garbage_collect_fails_to_run_delete_on_one_index() { let storage_resolver = StorageResolver::unconfigured(); - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) - .returning(move |_list_indexes_query: ListIndexesQuery| { - Ok(vec![ + .returning(|_list_indexes_request| { + let indexes_metadata = vec![ IndexMetadata::for_test("test-index-1", "ram://indexes/test-index-1"), IndexMetadata::for_test("test-index-2", "ram://indexes/test-index-2"), - ]) + ]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore .expect_list_splits() .times(4) - .returning(|query| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert!(["test-index-1", "test-index-2"].contains(&query.index_uids[0].index_id())); let splits = match query.split_states[0] { SplitState::Staged => make_splits(&["a"], SplitState::Staged), @@ -625,21 +694,28 @@ mod tests { } _ => panic!("only Staged and MarkedForDeletion expected."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .times(2) - .returning(|index_uid, split_ids| { + .returning(|mark_splits_for_deletion_request| { + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.clone().into(); assert!(["test-index-1", "test-index-2"].contains(&index_uid.index_id())); - assert_eq!(split_ids, vec!["a"]); - Ok(()) + assert_eq!(mark_splits_for_deletion_request.split_ids, vec!["a"]); + Ok(EmptyResponse {}) }); mock_metastore .expect_delete_splits() .times(2) - .returning(|index_uid, split_ids| { - let split_ids = HashSet::<&str>::from_iter(split_ids.iter().copied()); + .returning(|delete_splits_request| { + let index_uid: IndexUid = delete_splits_request.index_uid.clone().into(); + let split_ids = HashSet::<&str>::from_iter( + delete_splits_request + .split_ids + .iter() + .map(|split_id| split_id.as_str()), + ); let expected_split_ids = HashSet::<&str>::from_iter(["a", "b"]); assert_eq!(split_ids, expected_split_ids); @@ -652,12 +728,14 @@ mod tests { message: "fail to delete".to_string(), }) } else { - Ok(()) + Ok(EmptyResponse {}) } }); - let garbage_collect_actor = - GarbageCollector::new(Arc::new(mock_metastore), storage_resolver); + let garbage_collect_actor = GarbageCollector::new( + MetastoreServiceClient::from(mock_metastore), + storage_resolver, + ); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(garbage_collect_actor); diff --git a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs index 80279b3f1a4..ad82fc6b218 100644 --- a/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs +++ b/quickwit/quickwit-janitor/src/actors/retention_policy_executor.rs @@ -18,14 +18,16 @@ // along with this program. If not, see . use std::collections::{HashMap, HashSet}; -use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, Handler}; use quickwit_config::IndexConfig; -use quickwit_metastore::{ListIndexesQuery, Metastore}; +use quickwit_metastore::{ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt}; +use quickwit_proto::metastore::{ + ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::IndexUid; use serde::Serialize; use tracing::{debug, error, info}; @@ -58,7 +60,7 @@ struct Execute { /// It keeps a list of indexes that have retention policy configured /// in a cache and periodically update this list. pub struct RetentionPolicyExecutor { - metastore: Arc, + metastore: MetastoreServiceClient, /// A map of index_id to index metadata that are managed by this executor. /// This act as local cache that is periodically updated while taking into /// account deleted indexes, updated or removed retention policy on indexes. @@ -67,7 +69,7 @@ pub struct RetentionPolicyExecutor { } impl RetentionPolicyExecutor { - pub fn new(metastore: Arc) -> Self { + pub fn new(metastore: MetastoreServiceClient) -> Self { Self { metastore, index_configs: HashMap::new(), @@ -83,8 +85,9 @@ impl RetentionPolicyExecutor { let index_metadatas = match self .metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .and_then(|response| response.deserialize_indexes_metadata()) { Ok(metadatas) => metadatas, Err(error) => { @@ -253,7 +256,11 @@ mod tests { use quickwit_actors::Universe; use quickwit_config::RetentionPolicy; use quickwit_metastore::{ - IndexMetadata, ListIndexesQuery, MockMetastore, Split, SplitMetadata, SplitState, + IndexMetadata, ListSplitsRequestExt, ListSplitsResponseExt, Split, SplitMetadata, + SplitState, + }; + use quickwit_proto::metastore::{ + EmptyResponse, ListIndexesMetadataResponse, ListSplitsResponse, }; use super::*; @@ -336,50 +343,63 @@ mod tests { #[tokio::test] async fn test_retention_executor_refresh() -> anyhow::Result<()> { - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); let mut sequence = Sequence::new(); mock_metastore .expect_list_splits() .times(..) - .returning(|_| Ok(Vec::new())); + .returning(|_| Ok(ListSplitsResponse::empty())); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) .in_sequence(&mut sequence) - .returning(|_list_indexes_query: ListIndexesQuery| { - Ok(make_indexes(&[ - ("a", Some("1 hour")), - ("b", Some("1 hour")), - ("c", None), - ])) + .returning(|_list_indexes_request| { + let indexes_metadata = make_indexes(&[ + ("index-1", Some("1 hour")), + ("index-2", Some("1 hour")), + ("index-3", None), + ]); + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) .in_sequence(&mut sequence) - .returning(|_list_indexes_query: ListIndexesQuery| { - Ok(make_indexes(&[ - ("a", Some("1 hour")), - ("b", Some("2 hour")), - ("c", Some("1 hour")), - ])) + .returning(|_list_indexes_request| { + let indexes_metadata = make_indexes(&[ + ("index-1", Some("1 hour")), + ("index-2", Some("2 hour")), + ("index-3", Some("1 hour")), + ]); + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(1) .in_sequence(&mut sequence) - .returning(|_list_indexes_query: ListIndexesQuery| { - Ok(make_indexes(&[ - ("b", Some("1 hour")), - ("d", Some("1 hour")), - ("e", None), - ])) + .returning(|_list_indexes_request| { + let indexes_metadata = make_indexes(&[ + ("index-2", Some("1 hour")), + ("index-4", Some("1 hour")), + ("index-5", None), + ]); + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); - let retention_policy_executor = RetentionPolicyExecutor::new(Arc::new(mock_metastore)); + let retention_policy_executor = + RetentionPolicyExecutor::new(MetastoreServiceClient::from(mock_metastore)); let universe = Universe::with_accelerated_time(); let (mailbox, handle) = universe.spawn_builder().spawn(retention_policy_executor); @@ -387,8 +407,8 @@ mod tests { assert_eq!(counters.num_refresh_passes, 1); mailbox .ask(AssertState(vec![ - ("a", Some("1 hour")), - ("b", Some("1 hour")), + ("index-1", Some("1 hour")), + ("index-2", Some("1 hour")), ])) .await?; @@ -397,9 +417,9 @@ mod tests { assert_eq!(counters.num_refresh_passes, 2); mailbox .ask(AssertState(vec![ - ("a", Some("1 hour")), - ("b", Some("2 hour")), - ("c", Some("1 hour")), + ("index-1", Some("1 hour")), + ("index-2", Some("2 hour")), + ("index-3", Some("1 hour")), ])) .await?; @@ -408,8 +428,8 @@ mod tests { assert_eq!(counters.num_refresh_passes, 3); mailbox .ask(AssertState(vec![ - ("b", Some("1 hour")), - ("d", Some("1 hour")), + ("index-2", Some("1 hour")), + ("index-4", Some("1 hour")), ])) .await?; universe.assert_quit().await; @@ -419,47 +439,57 @@ mod tests { #[tokio::test] async fn test_retention_policy_execution_calls_dependencies() -> anyhow::Result<()> { - let mut mock_metastore = MockMetastore::default(); + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .times(..) - .returning(|_list_indexes_query: ListIndexesQuery| { - Ok(make_indexes(&[ - ("a", Some("2 hour")), - ("b", Some("1 hour")), - ("c", None), - ])) + .returning(|_list_indexes_request| { + let indexes_metadata = make_indexes(&[ + ("index-1", Some("2 hour")), + ("index-2", Some("1 hour")), + ("index-3", None), + ]); + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); mock_metastore .expect_list_splits() .times(2..=4) - .returning(|query| { + .returning(|list_splits_request| { + let query = list_splits_request.deserialize_list_splits_query().unwrap(); assert_eq!(query.split_states, &[SplitState::Published]); let splits = match query.index_uids[0].index_id() { - "a" => { + "index-1" => { vec![ make_split("split-1", Some(1000..=5000)), make_split("split-2", Some(2000..=6000)), make_split("split-3", None), ] } - "b" => Vec::new(), + "index-2" => Vec::new(), unknown => panic!("Unknown index: `{unknown}`."), }; - Ok(splits) + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); mock_metastore .expect_mark_splits_for_deletion() .times(1..=3) - .returning(|index_uid, split_ids| { - assert_eq!(index_uid.index_id(), "a"); - assert_eq!(split_ids, ["split-1", "split-2"]); - Ok(()) + .returning(|mark_splits_for_deletion_request| { + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.clone().into(); + assert_eq!(index_uid.index_id(), "index-1"); + assert_eq!( + mark_splits_for_deletion_request.split_ids, + ["split-1", "split-2"] + ); + Ok(EmptyResponse {}) }); - let retention_policy_executor = RetentionPolicyExecutor::new(Arc::new(mock_metastore)); + let retention_policy_executor = + RetentionPolicyExecutor::new(MetastoreServiceClient::from(mock_metastore)); let universe = Universe::with_accelerated_time(); let (_mailbox, handle) = universe.spawn_builder().spawn(retention_policy_executor); diff --git a/quickwit/quickwit-janitor/src/lib.rs b/quickwit/quickwit-janitor/src/lib.rs index 906a7902277..84b4a3cf21c 100644 --- a/quickwit/quickwit-janitor/src/lib.rs +++ b/quickwit/quickwit-janitor/src/lib.rs @@ -19,12 +19,11 @@ #![deny(clippy::disallowed_methods)] -use std::sync::Arc; - use quickwit_actors::{Mailbox, Universe}; use quickwit_common::pubsub::EventBroker; use quickwit_config::NodeConfig; -use quickwit_metastore::{Metastore, SplitInfo}; +use quickwit_metastore::SplitInfo; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_search::SearchJobPlacer; use quickwit_storage::StorageResolver; use tracing::info; @@ -47,7 +46,7 @@ pub struct JanitorApiSchemas; pub async fn start_janitor_service( universe: &Universe, config: &NodeConfig, - metastore: Arc, + metastore: MetastoreServiceClient, search_job_placer: SearchJobPlacer, storage_resolver: StorageResolver, event_broker: EventBroker, diff --git a/quickwit/quickwit-janitor/src/retention_policy_execution.rs b/quickwit/quickwit-janitor/src/retention_policy_execution.rs index 972af691f90..aee0f776935 100644 --- a/quickwit/quickwit-janitor/src/retention_policy_execution.rs +++ b/quickwit/quickwit-janitor/src/retention_policy_execution.rs @@ -17,13 +17,16 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; - use quickwit_actors::ActorContext; use quickwit_common::PrettySample; use quickwit_config::RetentionPolicy; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitMetadata, SplitState}; -use quickwit_proto::IndexUid; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, +}; +use quickwit_proto::metastore::{ + ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreService, MetastoreServiceClient, +}; +use quickwit_proto::{IndexUid, SplitId}; use time::OffsetDateTime; use tracing::{info, warn}; @@ -39,7 +42,7 @@ use crate::actors::RetentionPolicyExecutor; /// * `ctx_opt` - A context for reporting progress (only useful within quickwit actor). pub async fn run_execute_retention_policy( index_uid: IndexUid, - metastore: Arc, + mut metastore: MetastoreServiceClient, retention_policy: &RetentionPolicy, ctx: &ActorContext, ) -> anyhow::Result> { @@ -51,9 +54,11 @@ pub async fn run_execute_retention_policy( .with_split_state(SplitState::Published) .with_time_range_end_lte(max_retention_timestamp); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; let (expired_splits, ignored_splits): (Vec, Vec) = ctx - .protect_future(metastore.list_splits(query)) + .protect_future(metastore.list_splits(list_splits_request)) .await? + .deserialize_splits()? .into_iter() .map(|split| split.split_metadata) .partition(|split_metadata| split_metadata.time_range.is_some()); @@ -74,9 +79,9 @@ pub async fn run_execute_retention_policy( return Ok(expired_splits); } // Mark the expired splits for deletion. - let expired_split_ids: Vec<&str> = expired_splits + let expired_split_ids: Vec = expired_splits .iter() - .map(|split_metadata| split_metadata.split_id()) + .map(|split_metadata| split_metadata.split_id.to_string()) .collect(); info!( index_id=%index_uid.index_id(), @@ -84,7 +89,9 @@ pub async fn run_execute_retention_policy( "Marking {} splits for deletion based on retention policy.", expired_split_ids.len() ); - ctx.protect_future(metastore.mark_splits_for_deletion(index_uid, &expired_split_ids)) + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), expired_split_ids); + ctx.protect_future(metastore.mark_splits_for_deletion(mark_splits_for_deletion_request)) .await?; Ok(expired_splits) } diff --git a/quickwit/quickwit-metastore-utils/src/bin/proxy.rs b/quickwit/quickwit-metastore-utils/src/bin/proxy.rs index 57ffb14fa64..7712b63f50e 100644 --- a/quickwit/quickwit-metastore-utils/src/bin/proxy.rs +++ b/quickwit/quickwit-metastore-utils/src/bin/proxy.rs @@ -23,10 +23,6 @@ use std::sync::Arc; use async_trait::async_trait; use quickwit_metastore_utils::{GrpcCall, GrpcRequest}; -use quickwit_proto::metastore::metastore_service_client::MetastoreServiceClient; -use quickwit_proto::metastore::metastore_service_server::{ - MetastoreService, MetastoreServiceServer, -}; use quickwit_proto::metastore::*; use quickwit_proto::tonic; use quickwit_proto::tonic::transport::Channel; @@ -39,7 +35,7 @@ use tokio::time::Instant; struct Inner { start: Instant, - client: MetastoreServiceClient, + client: MetastoreServiceClient, file: BufWriter, } @@ -48,7 +44,7 @@ struct MetastoreProxyService { } impl MetastoreProxyService { - pub fn new(client: MetastoreServiceClient, record_file: File) -> Self { + pub fn new(client: MetastoreServiceClient, record_file: File) -> Self { let inner = Inner { start: Instant::now(), client, @@ -99,13 +95,13 @@ impl MetastoreService for MetastoreProxyService { Ok(resp) } /// Gets an indexes metadatas. - async fn list_indexes_metadatas( + async fn list_indexes_metadata( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { let mut lock = self.inner.lock().await; lock.record(request.get_ref().clone()).await.unwrap(); - let resp = lock.client.list_indexes_metadatas(request).await?; + let resp = lock.client.list_indexes_metadata(request).await?; Ok(resp) } /// Deletes an index @@ -118,16 +114,6 @@ impl MetastoreService for MetastoreProxyService { let resp = lock.client.delete_index(request).await?; Ok(resp) } - /// Gets all splits from index. - async fn list_all_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - let mut lock = self.inner.lock().await; - lock.record(request.get_ref().clone()).await.unwrap(); - let resp = lock.client.list_all_splits(request).await?; - Ok(resp) - } /// Gets splits from index. async fn list_splits( &self, diff --git a/quickwit/quickwit-metastore-utils/src/bin/replay.rs b/quickwit/quickwit-metastore-utils/src/bin/replay.rs index 1b22b088605..82715830ae5 100644 --- a/quickwit/quickwit-metastore-utils/src/bin/replay.rs +++ b/quickwit/quickwit-metastore-utils/src/bin/replay.rs @@ -37,15 +37,12 @@ async fn replay_grpc_request( GrpcRequest::IndexMetadataRequest(req) => { client.index_metadata(req).await?; } - GrpcRequest::ListIndexesMetadatasRequest(req) => { - client.list_indexes_metadatas(req).await?; + GrpcRequest::ListIndexesMetadataRequest(req) => { + client.list_indexes_metadata(req).await?; } GrpcRequest::DeleteIndexRequest(req) => { client.delete_index(req).await?; } - GrpcRequest::ListAllSplitsRequest(req) => { - client.list_all_splits(req).await?; - } GrpcRequest::ListSplitsRequest(req) => { client.list_splits(req).await?; } diff --git a/quickwit/quickwit-metastore-utils/src/grpc_request.rs b/quickwit/quickwit-metastore-utils/src/grpc_request.rs index 15c612a3ca4..d818ee1c80c 100644 --- a/quickwit/quickwit-metastore-utils/src/grpc_request.rs +++ b/quickwit/quickwit-metastore-utils/src/grpc_request.rs @@ -67,9 +67,8 @@ macro_rules! req_from_impls { generate_req_enum!( CreateIndexRequest, IndexMetadataRequest, - ListIndexesMetadatasRequest, + ListIndexesMetadataRequest, DeleteIndexRequest, - ListAllSplitsRequest, ListSplitsRequest, StageSplitsRequest, PublishSplitsRequest, diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index eb5762b6df4..9081c7e74d4 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -36,7 +36,6 @@ mod error; mod metastore; mod metastore_factory; mod metastore_resolver; -mod metrics; mod split_metadata; mod split_metadata_version; @@ -45,16 +44,14 @@ use std::ops::Range; pub use error::MetastoreResolverError; pub use metastore::control_plane_metastore::ControlPlaneMetastore; pub use metastore::file_backed_metastore::FileBackedMetastore; -pub use metastore::grpc_metastore::{GrpcMetastoreAdapter, MetastoreGrpcClient}; pub(crate) use metastore::index_metadata::serialize::{IndexMetadataV0_6, VersionedIndexMetadata}; -pub use metastore::metastore_event_publisher::MetastoreEventPublisher; #[cfg(feature = "postgres")] pub use metastore::postgresql_metastore::PostgresqlMetastore; -pub use metastore::retrying_metastore::RetryingMetastore; -#[cfg(any(test, feature = "testsuite"))] -pub use metastore::MockMetastore; pub use metastore::{ - file_backed_metastore, IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, + file_backed_metastore, AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, + IndexMetadataResponseExt, ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, + ListIndexesQuery, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, + MetastoreServiceExt, PublishSplitsRequestExt, StageSplitsRequestExt, }; pub use metastore_factory::{MetastoreFactory, UnsupportedMetastore}; pub use metastore_resolver::MetastoreResolver; @@ -77,20 +74,25 @@ pub struct MetastoreApiSchemas; /// Returns `true` if the split time range is included in `time_range_opt`. /// If `time_range_opt` is None, returns always true. -pub fn split_time_range_filter(split: &Split, time_range_opt: Option<&Range>) -> bool { - match (time_range_opt, split.split_metadata.time_range.as_ref()) { +pub fn split_time_range_filter( + split_metadata: &SplitMetadata, + time_range_opt: Option<&Range>, +) -> bool { + match (time_range_opt, split_metadata.time_range.as_ref()) { (Some(filter_time_range), Some(split_time_range)) => { !is_disjoint(filter_time_range, split_time_range) } _ => true, // Return `true` if `time_range` is omitted or the split has no time range. } } - /// Returns `true` if the tags filter evaluation is true. /// If `tags_filter_opt` is None, returns always true. -pub fn split_tag_filter(split: &Split, tags_filter_opt: Option<&TagFilterAst>) -> bool { +pub fn split_tag_filter( + split_metadata: &SplitMetadata, + tags_filter_opt: Option<&TagFilterAst>, +) -> bool { tags_filter_opt - .map(|tags_filter_ast| tags_filter_ast.evaluate(&split.split_metadata.tags)) + .map(|tags_filter_ast| tags_filter_ast.evaluate(&split_metadata.tags)) .unwrap_or(true) } @@ -101,13 +103,14 @@ mod backward_compatibility_tests; mod for_test { use std::sync::Arc; + use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_storage::RamStorage; - use super::{FileBackedMetastore, Metastore}; + use super::FileBackedMetastore; /// Returns a metastore backed by an "in-memory file" for testing. - pub fn metastore_for_test() -> Arc { - Arc::new(FileBackedMetastore::for_test(Arc::new( + pub fn metastore_for_test() -> MetastoreServiceClient { + MetastoreServiceClient::new(FileBackedMetastore::for_test(Arc::new( RamStorage::default(), ))) } diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 1280504ec3d..762dc14838c 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -18,29 +18,30 @@ // along with this program. If not, see . use std::fmt; -use std::sync::Arc; use async_trait::async_trait; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_proto::control_plane::{ControlPlaneService, ControlPlaneServiceClient}; use quickwit_proto::metastore::{ - serde_utils as metastore_serde_utils, AcquireShardsRequest, AcquireShardsResponse, - AddSourceRequest, CloseShardsRequest, CloseShardsResponse, CreateIndexRequest, - DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, - DeleteSourceRequest, DeleteTask, ListShardsRequest, ListShardsResponse, MetastoreResult, - OpenShardsRequest, OpenShardsResponse, ToggleSourceRequest, + AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CloseShardsRequest, + CloseShardsResponse, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteShardsRequest, DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, IndexMetadataRequest, IndexMetadataResponse, + LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, + ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, + ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, + MetastoreServiceClient, OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, + ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; -use quickwit_proto::{IndexUid, PublishToken}; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata}; - -/// A [`Metastore`] implementation that proxies some requests to the control plane so it can +/// A [`MetastoreService`] implementation that proxies some requests to the control plane so it can /// track the state of the metastore accurately and react to events in real-time. +#[derive(Clone)] pub struct ControlPlaneMetastore { control_plane: ControlPlaneServiceClient, - metastore: Arc, + metastore: MetastoreServiceClient, } impl fmt::Debug for ControlPlaneMetastore { @@ -51,7 +52,10 @@ impl fmt::Debug for ControlPlaneMetastore { impl ControlPlaneMetastore { /// Creates a new [`ControlPlaneMetastore`]. - pub fn new(control_plane: ControlPlaneServiceClient, metastore: Arc) -> Self { + pub fn new( + control_plane: ControlPlaneServiceClient, + metastore: MetastoreServiceClient, + ) -> Self { Self { control_plane, metastore, @@ -60,197 +64,181 @@ impl ControlPlaneMetastore { } #[async_trait] -impl Metastore for ControlPlaneMetastore { - fn uri(&self) -> &Uri { - self.metastore.uri() +impl MetastoreService for ControlPlaneMetastore { + fn endpoints(&self) -> Vec { + self.metastore.endpoints() } - async fn check_connectivity(&self) -> anyhow::Result<()> { + async fn check_connectivity(&mut self) -> anyhow::Result<()> { self.metastore.check_connectivity().await } // Proxied metastore API calls. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_config_json = metastore_serde_utils::to_json_str(&index_config)?; - let request = CreateIndexRequest { index_config_json }; - let response = self.control_plane.clone().create_index(request).await?; - let index_uid: IndexUid = response.index_uid.into(); - Ok(index_uid) + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let response = self.control_plane.create_index(request).await?; + Ok(response) } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - let request = DeleteIndexRequest { - index_uid: index_uid.into(), - }; - self.control_plane.clone().delete_index(request).await?; - Ok(()) + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> MetastoreResult { + let response = self.control_plane.delete_index(request).await?; + Ok(response) } - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - let request = AddSourceRequest { - index_uid: index_uid.into(), - source_config_json: metastore_serde_utils::to_json_str(&source)?, - }; - self.control_plane.clone().add_source(request).await?; - Ok(()) + async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + let response = self.control_plane.add_source(request).await?; + Ok(response) } async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - let request = ToggleSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - enable, - }; - self.control_plane.clone().toggle_source(request).await?; - Ok(()) + &mut self, + request: ToggleSourceRequest, + ) -> MetastoreResult { + let response = self.control_plane.clone().toggle_source(request).await?; + Ok(response) } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - let request = DeleteSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - }; - self.control_plane.clone().delete_source(request).await?; - Ok(()) + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> MetastoreResult { + let response = self.control_plane.delete_source(request).await?; + Ok(response) } // Other metastore API calls. - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - self.metastore.index_metadata(index_id).await + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + self.metastore.index_metadata(request).await } - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { - self.metastore.list_indexes_metadatas(query).await + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> MetastoreResult { + self.metastore.list_indexes_metadata(request).await } async fn stage_splits( - &self, - index_uid: IndexUid, - splits_metadata: Vec, - ) -> MetastoreResult<()> { - self.metastore - .stage_splits(index_uid, splits_metadata) - .await + &mut self, + request: StageSplitsRequest, + ) -> MetastoreResult { + self.metastore.stage_splits(request).await + } + + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> MetastoreResult { + self.metastore.publish_splits(request).await } - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - self.metastore - .publish_splits( - index_uid, - staged_split_ids, - replaced_split_ids, - checkpoint_delta_opt, - publish_token_opt, - ) - .await + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> MetastoreResult { + self.metastore.list_splits(request).await } - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - self.metastore.list_splits(query).await + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> MetastoreResult { + self.metastore.list_stale_splits(request).await } - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.metastore - .mark_splits_for_deletion(index_uid, split_ids) - .await + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + self.metastore.mark_splits_for_deletion(request).await } - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.metastore.delete_splits(index_uid, split_ids).await + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> MetastoreResult { + self.metastore.delete_splits(request).await } async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - self.metastore - .reset_source_checkpoint(index_uid, source_id) - .await + &mut self, + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + self.metastore.reset_source_checkpoint(request).await } // Delete tasks API - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { + async fn create_delete_task( + &mut self, + delete_query: DeleteQuery, + ) -> MetastoreResult { self.metastore.create_delete_task(delete_query).await } - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - self.metastore.last_delete_opstamp(index_uid).await + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> MetastoreResult { + self.metastore.last_delete_opstamp(request).await } - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - self.metastore - .update_splits_delete_opstamp(index_uid, split_ids, delete_opstamp) - .await + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + self.metastore.update_splits_delete_opstamp(request).await } async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - self.metastore - .list_delete_tasks(index_uid, opstamp_start) - .await + &mut self, + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + self.metastore.list_delete_tasks(request).await } // Shard API - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> MetastoreResult { self.metastore.open_shards(request).await } async fn acquire_shards( - &self, + &mut self, request: AcquireShardsRequest, ) -> MetastoreResult { self.metastore.acquire_shards(request).await } - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> MetastoreResult { self.metastore.list_shards(request).await } async fn close_shards( - &self, + &mut self, request: CloseShardsRequest, ) -> MetastoreResult { self.metastore.close_shards(request).await } async fn delete_shards( - &self, + &mut self, request: DeleteShardsRequest, ) -> MetastoreResult { self.metastore.delete_shards(request).await diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs index 69f6f992bd3..0c7a23718db 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs @@ -28,7 +28,7 @@ use std::collections::HashMap; use std::fmt::Debug; use std::ops::Bound; -use itertools::Either; +use itertools::{Either, Itertools}; use quickwit_common::PrettySample; use quickwit_config::{SourceConfig, INGEST_SOURCE_ID}; use quickwit_proto::metastore::{ @@ -247,8 +247,8 @@ impl FileBackedIndex { /// Marks the splits for deletion. Returns whether a mutation occurred. pub(crate) fn mark_splits_for_deletion( &mut self, - split_ids: &[&str], - deletable_states: &[SplitState], + split_ids: impl IntoIterator>, + deletable_split_states: &[SplitState], return_error_on_splits_not_found: bool, ) -> MetastoreResult { let mut mutation_occurred = false; @@ -256,24 +256,24 @@ impl FileBackedIndex { let mut non_deletable_split_ids = Vec::new(); let now_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - for &split_id in split_ids { + for split_id in split_ids { + let split_id_ref = split_id.as_ref(); // Check for the existence of split. - let metadata = match self.splits.get_mut(split_id) { + let metadata = match self.splits.get_mut(split_id_ref) { Some(metadata) => metadata, None => { - split_not_found_ids.push(split_id.to_string()); + split_not_found_ids.push(split_id_ref.to_string()); continue; } }; - if !deletable_states.contains(&metadata.split_state) { - non_deletable_split_ids.push(split_id.to_string()); + if !deletable_split_states.contains(&metadata.split_state) { + non_deletable_split_ids.push(split_id_ref.to_string()); continue; }; if metadata.split_state == SplitState::MarkedForDeletion { // If the split is already marked for deletion, This is fine, we just skip it. continue; } - metadata.split_state = SplitState::MarkedForDeletion; metadata.update_timestamp = now_timestamp; mutation_occurred = true; @@ -304,14 +304,20 @@ impl FileBackedIndex { /// Helper to mark a list of splits as published. /// This function however does not update the checkpoint. - fn mark_splits_as_published_helper(&mut self, split_ids: &[&str]) -> MetastoreResult<()> { + fn mark_splits_as_published_helper( + &mut self, + staged_split_ids: impl IntoIterator>, + ) -> MetastoreResult<()> { let mut split_not_found_ids = Vec::new(); let mut split_not_staged_ids = Vec::new(); + let now_timestamp = OffsetDateTime::now_utc().unix_timestamp(); - for &split_id in split_ids { + + for staged_plit_id in staged_split_ids { + let staged_split_id_ref = staged_plit_id.as_ref(); // Check for the existence of split. - let Some(metadata) = self.splits.get_mut(split_id) else { - split_not_found_ids.push(split_id.to_string()); + let Some(metadata) = self.splits.get_mut(staged_split_id_ref) else { + split_not_found_ids.push(staged_split_id_ref.to_string()); continue; }; if metadata.split_state == SplitState::Staged { @@ -319,16 +325,14 @@ impl FileBackedIndex { metadata.update_timestamp = now_timestamp; metadata.publish_timestamp = Some(now_timestamp); } else { - split_not_staged_ids.push(split_id.to_string()); + split_not_staged_ids.push(staged_split_id_ref.to_string()); } } - if !split_not_found_ids.is_empty() { return Err(MetastoreError::NotFound(EntityKind::Splits { split_ids: split_not_found_ids, })); } - if !split_not_staged_ids.is_empty() { let entity = EntityKind::Splits { split_ids: split_not_staged_ids, @@ -336,15 +340,14 @@ impl FileBackedIndex { let message = "splits are not staged".to_string(); return Err(MetastoreError::FailedPrecondition { entity, message }); } - Ok(()) } /// Publishes splits. - pub(crate) fn publish_splits<'a>( + pub(crate) fn publish_splits( &mut self, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], + staged_split_ids: impl IntoIterator>, + replaced_split_ids: impl IntoIterator>, checkpoint_delta_opt: Option, publish_token_opt: Option, ) -> MetastoreResult<()> { @@ -383,15 +386,34 @@ impl FileBackedIndex { let limit = query.limit.unwrap_or(usize::MAX); let offset = query.offset.unwrap_or_default(); - let splits: Vec = self - .splits - .values() - .filter(|split| split_query_predicate(split, query)) - .skip(offset) - .take(limit) - .cloned() - .collect(); - + let splits: Vec = if query.sort_by_staleness { + self.splits + .values() + .filter(|split| split_query_predicate(split, query)) + .sorted_unstable_by(|left_split, right_split| { + left_split + .split_metadata + .delete_opstamp + .cmp(&right_split.split_metadata.delete_opstamp) + .then_with(|| { + left_split + .publish_timestamp + .cmp(&right_split.publish_timestamp) + }) + }) + .skip(offset) + .take(limit) + .cloned() + .collect() + } else { + self.splits + .values() + .filter(|split| split_query_predicate(split, query)) + .skip(offset) + .take(limit) + .cloned() + .collect() + }; Ok(splits) } @@ -408,18 +430,23 @@ impl FileBackedIndex { } /// Deletes multiple splits. - pub(crate) fn delete_splits(&mut self, split_ids: &[&str]) -> MetastoreResult<()> { + pub(crate) fn delete_splits( + &mut self, + split_ids: impl IntoIterator>, + ) -> MetastoreResult<()> { + let num_deleted_splits = 0; let mut split_not_found_ids = Vec::new(); let mut split_not_deletable_ids = Vec::new(); - for &split_id in split_ids { - match self.delete_split(split_id) { + for split_id in split_ids { + let split_id_ref = split_id.as_ref(); + match self.delete_split(split_id_ref) { DeleteSplitOutcome::Success => {} DeleteSplitOutcome::SplitNotFound => { - split_not_found_ids.push(split_id); + split_not_found_ids.push(split_id_ref.to_string()); } DeleteSplitOutcome::Forbidden => { - split_not_deletable_ids.push(split_id.to_string()); + split_not_deletable_ids.push(split_id_ref.to_string()); } } } @@ -430,11 +457,11 @@ impl FileBackedIndex { let message = "splits are not deletable".to_string(); return Err(MetastoreError::FailedPrecondition { entity, message }); } - info!(index_id=%self.index_id(), "Deleted {} splits from index.", split_ids.len()); + info!(index_id=%self.index_id(), "Deleted {num_deleted_splits} splits from index."); if !split_not_found_ids.is_empty() { warn!( - index_id=%self.index_id(), + index_id=self.index_id().to_string(), split_ids=?PrettySample::new(&split_not_found_ids, 5), "{} splits were not found and could not be deleted.", split_not_found_ids.len() @@ -682,7 +709,7 @@ impl Debug for Stamper { } fn split_query_predicate(split: &&Split, query: &ListSplitsQuery) -> bool { - if !split_tag_filter(split, query.tags.as_ref()) { + if !split_tag_filter(&split.split_metadata, query.tags.as_ref()) { return false; } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_metastore_factory.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_metastore_factory.rs index 10a34275791..6b174f7bc9a 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_metastore_factory.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_metastore_factory.rs @@ -18,21 +18,21 @@ // along with this program. If not, see . use std::collections::HashMap; -use std::sync::{Arc, Weak}; +use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; use once_cell::sync::OnceCell; use quickwit_common::uri::Uri; use quickwit_config::{MetastoreBackend, MetastoreConfig}; -use quickwit_proto::metastore::MetastoreError; +use quickwit_proto::metastore::{MetastoreError, MetastoreServiceClient}; use quickwit_storage::{StorageResolver, StorageResolverError}; use regex::Regex; use tokio::sync::Mutex; use tracing::debug; -use crate::metastore::instrumented_metastore::InstrumentedMetastore; -use crate::{FileBackedMetastore, Metastore, MetastoreFactory, MetastoreResolverError}; +use crate::metastore::instrument_metastore; +use crate::{FileBackedMetastore, MetastoreFactory, MetastoreResolverError}; /// A file-backed metastore factory. /// @@ -44,11 +44,9 @@ use crate::{FileBackedMetastore, Metastore, MetastoreFactory, MetastoreResolverE #[derive(Clone)] pub struct FileBackedMetastoreFactory { storage_resolver: StorageResolver, - // We almost never garbage collect the dangling Weak pointers - // here. This is judged to not be much of a problem however. - // - // In a normal run, this cache will contain a single Metastore. - cache: Arc>>>, + // We never garbage collect unused metastore client instances. This should not be a problem + // because during normal use this cache will hold at most a single instance. + cache: Arc>>, } fn extract_polling_interval_from_uri(uri: &str) -> (String, Option) { @@ -78,9 +76,8 @@ impl FileBackedMetastoreFactory { } } - async fn get_from_cache(&self, uri: &Uri) -> Option> { - let cache_lock = self.cache.lock().await; - cache_lock.get(uri).and_then(Weak::upgrade) + async fn get_from_cache(&self, uri: &Uri) -> Option { + self.cache.lock().await.get(uri).cloned() } /// If there is a valid entry in the cache to begin with, we ignore the new @@ -88,15 +85,17 @@ impl FileBackedMetastoreFactory { /// /// This way we make sure that we keep only one instance associated /// to the key `uri` outside of this struct. - async fn cache_metastore(&self, uri: Uri, metastore: Arc) -> Arc { - let mut cache_lock = self.cache.lock().await; - if let Some(metastore_weak) = cache_lock.get(&uri) { - if let Some(metastore_arc) = metastore_weak.upgrade() { - return metastore_arc.clone(); - } - } - cache_lock.insert(uri, Arc::downgrade(&metastore)); - metastore + async fn cache_metastore( + &self, + uri: Uri, + metastore: MetastoreServiceClient, + ) -> MetastoreServiceClient { + self.cache + .lock() + .await + .entry(uri) + .or_insert(metastore) + .clone() } } @@ -110,7 +109,7 @@ impl MetastoreFactory for FileBackedMetastoreFactory { &self, _metastore_config: &MetastoreConfig, uri: &Uri, - ) -> Result, MetastoreResolverError> { + ) -> Result { let (uri_stripped, polling_interval_opt) = extract_polling_interval_from_uri(uri.as_str()); let uri = Uri::from_well_formed(uri_stripped); if let Some(metastore) = self.get_from_cache(&uri).await { @@ -142,10 +141,8 @@ impl MetastoreFactory for FileBackedMetastoreFactory { let file_backed_metastore = FileBackedMetastore::try_new(storage, polling_interval_opt) .await .map_err(MetastoreResolverError::Initialization)?; - let instrumented_metastore = InstrumentedMetastore::new(Box::new(file_backed_metastore)); - let unique_metastore_for_uri = self - .cache_metastore(uri, Arc::new(instrumented_metastore)) - .await; + let instrumented_metastore = instrument_metastore(file_backed_metastore); + let unique_metastore_for_uri = self.cache_metastore(uri, instrumented_metastore).await; Ok(unique_metastore_for_uri) } } 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 e9769b48e19..4d926c99f5a 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -26,26 +26,33 @@ mod file_backed_metastore_factory; mod lazy_file_backed_index; mod store_operations; +use core::fmt; use std::collections::HashMap; -use std::fmt; use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; use futures::future::try_join_all; use itertools::{Either, Itertools}; -use quickwit_common::uri::Uri; -use quickwit_config::{validate_index_id_pattern, IndexConfig, SourceConfig}; +use quickwit_config::validate_index_id_pattern; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, AcquireShardsSubrequest, CloseShardsRequest, - CloseShardsResponse, CloseShardsSubrequest, DeleteQuery, DeleteShardsRequest, - DeleteShardsResponse, DeleteShardsSubrequest, DeleteTask, EntityKind, ListShardsRequest, - ListShardsResponse, MetastoreError, MetastoreResult, OpenShardsRequest, OpenShardsResponse, - OpenShardsSubrequest, + AcquireShardsRequest, AcquireShardsResponse, AcquireShardsSubrequest, AddSourceRequest, + CloseShardsRequest, CloseShardsResponse, CloseShardsSubrequest, CreateIndexRequest, + CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteShardsSubrequest, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, EntityKind, IndexMetadataRequest, IndexMetadataResponse, + LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, + ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, + ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, + MetastoreService, OpenShardsRequest, OpenShardsResponse, OpenShardsSubrequest, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::IndexUid; use quickwit_storage::Storage; use regex::RegexSet; +use time::OffsetDateTime; use tokio::sync::{Mutex, OwnedMutexGuard, RwLock}; use self::file_backed_index::FileBackedIndex; @@ -55,9 +62,15 @@ use self::store_operations::{ check_indexes_states_exist, delete_index, fetch_index, fetch_or_init_indexes_states, index_exists, put_index, put_indexes_states, }; +use super::{ + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, + ListIndexesMetadataResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, + PublishSplitsRequestExt, StageSplitsRequestExt, +}; use crate::checkpoint::IndexCheckpointDelta; use crate::{ - IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata, SplitState, + IndexMetadata, ListIndexesMetadataRequestExt, ListIndexesQuery, ListSplitsQuery, + MetastoreServiceExt, SplitState, }; /// State of an index tracked by the metastore. @@ -118,6 +131,7 @@ impl From for MutationOccurred<()> { /// - single-node environment; /// - multiple-nodes environment with only one writer and readers. In this case, you must be very /// cautious and ensure that your readers are really readers. +#[derive(Clone)] pub struct FileBackedMetastore { storage: Arc, per_index_metastores: Arc>>, @@ -127,7 +141,7 @@ pub struct FileBackedMetastore { impl fmt::Debug for FileBackedMetastore { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.debug_struct("FileBackedMetastore") - .field("uri", self.storage.uri()) + .field("storage_uri", &self.storage.uri()) .field("polling_interval_opt", &self.polling_interval_opt) .finish() } @@ -275,7 +289,6 @@ impl FileBackedMetastore { return get_index_mutex(index_id, index_state).await; } } - // At this point we do not hold our mutex, so we need to do a little dance // to make sure we return the same instance. // @@ -313,18 +326,25 @@ impl FileBackedMetastore { async fn get_index(&self, index_uid: IndexUid) -> MetastoreResult { self.read(index_uid, |index| Ok(index.clone())).await } - - #[cfg(test)] - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - index_exists(&*self.storage, index_id).await - } } #[async_trait] -impl Metastore for FileBackedMetastore { +impl MetastoreService for FileBackedMetastore { + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + check_indexes_states_exist(self.storage.clone()).await + } + + fn endpoints(&self) -> Vec { + vec![self.storage.uri().clone()] + } + /// ------------------------------------------------------------------------------- /// Mutations over the high-level index. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_config = request.deserialize_index_config()?; let index_id = index_config.index_id.clone(); // We pick the outer lock here, so that we enter a critical section. @@ -381,13 +401,20 @@ impl Metastore for FileBackedMetastore { per_index_metastores_wlock.insert(index_id.clone(), IndexState::Creating); } put_res?; - Ok(index_uid) + let response = CreateIndexResponse { + index_uid: index_uid.into(), + }; + Ok(response) } - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> MetastoreResult { // We pick the outer lock here, so that we enter a critical section. let mut per_index_metastores_wlock = self.per_index_metastores.write().await; + let index_uid: IndexUid = request.index_uid.into(); let index_id = index_uid.index_id(); // If index is neither in `per_index_metastores_wlock` nor on the storage, it does not // exist. @@ -428,21 +455,23 @@ impl Metastore for FileBackedMetastore { }, _ => {} } - - delete_res + delete_res.map(|_| EmptyResponse {}) } /// ------------------------------------------------------------------------------- /// Mutations over a single index async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { + &mut self, + request: StageSplitsRequest, + ) -> MetastoreResult { + let splits_metadata = request.deserialize_splits_metadata()?; + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { let mut failed_split_ids = Vec::new(); - for split_metadata in split_metadata_list { + + for split_metadata in splits_metadata { match index.stage_split(split_metadata) { Ok(()) => {} Err(MetastoreError::FailedPrecondition { @@ -465,39 +494,38 @@ impl Metastore for FileBackedMetastore { } }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - self.mutate(index_uid, |index| { + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> MetastoreResult { + let index_checkpoint_delta: Option = + request.deserialize_index_checkpoint()?; + self.mutate(request.index_uid.into(), |index| { index.publish_splits( - staged_split_ids, - replaced_split_ids, - checkpoint_delta_opt, - publish_token_opt, + request.staged_split_ids, + request.replaced_split_ids, + index_checkpoint_delta, + request.publish_token_opt, )?; Ok(MutationOccurred::Yes(())) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index .mark_splits_for_deletion( - split_ids, + request.split_ids, &[ SplitState::Staged, SplitState::Published, @@ -508,97 +536,145 @@ impl Metastore for FileBackedMetastore { .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.delete_splits(split_ids)?; - Ok(MutationOccurred::Yes(())) + index.delete_splits(request.split_ids)?; + Ok(MutationOccurred::Yes(EmptyResponse {})) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.add_source(source)?; + index.add_source(source_config)?; Ok(MutationOccurred::Yes(())) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + &mut self, + request: ToggleSourceRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index - .toggle_source(source_id, enable) + .toggle_source(&request.source_id, request.enable) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { - index.delete_source(source_id).map(MutationOccurred::from) + index + .delete_source(&request.source_id) + .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + &mut self, + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { index - .reset_source_checkpoint(source_id) + .reset_source_checkpoint(&request.source_id) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(EmptyResponse {}) } /// ------------------------------------------------------------------------------- /// Read-only accessors - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { + 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 &query.index_uids { + for index_uid in &list_splits_query.index_uids { let splits = self - .read(index_uid.clone(), |index| index.list_splits(&query)) + .read(index_uid.clone(), |index| { + index.list_splits(&list_splits_query) + }) .await?; all_splits.extend(splits); } - Ok(all_splits) + let response = ListSplitsResponse::try_from_splits(all_splits)?; + Ok(response) } - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - self.read_any(index_id, |index| Ok(index.metadata().clone())) - .await + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> MetastoreResult { + let list_splits_query = ListSplitsQuery::for_index(request.index_uid.into()) + .with_delete_opstamp_lt(request.delete_opstamp) + .with_split_state(SplitState::Published) + .retain_mature(OffsetDateTime::now_utc()) + .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 } - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let index_id = request.get_index_id()?; + let index_metadata = self + .read_any(&index_id, |index| Ok(index.metadata().clone())) + .await?; + if let Some(index_uid) = &request.index_uid { + if index_metadata.index_uid.to_string() != *index_uid { + return Err(MetastoreError::NotFound(EntityKind::Index { + index_id: index_id.to_string(), + })); + } + } + let response = IndexMetadataResponse::try_from_index_metadata(index_metadata)?; + Ok(response) + } + + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> MetastoreResult { // Done in two steps: // 1) Get index IDs and release the lock on `per_index_metastores`. // 2) Get each index metadata. Note that each get will take a read lock on // `per_index_metastores`. Lock is released in 1) to let a concurrent task/thread to // take a write lock on `per_index_metastores`. + let query = request.deserialize_list_indexes_query()?; let index_matcher_result = match query { ListIndexesQuery::IndexIdPatterns(patterns) => build_regex_set_from_patterns(patterns), ListIndexesQuery::All => build_regex_set_from_patterns(vec!["*".to_string()]), @@ -620,41 +696,26 @@ impl Metastore for FileBackedMetastore { .cloned() .collect() }; - let indexes_metadatas: Vec = - try_join_all(index_ids.iter().map(|index_id| async move { - match self.index_metadata(index_id).await { - Ok(index_metadata) => Ok(Some(index_metadata)), - Err(MetastoreError::NotFound(EntityKind::Index { .. })) => Ok(None), - Err(MetastoreError::Internal { message, cause }) => { - // Indexes can be in a transition state `Creating` or `Deleting`. - // This is fine to ignore them. - if cause.contains("is in transitioning state") { - Ok(None) - } else { - Err(MetastoreError::Internal { message, cause }) - } - } - Err(error) => Err(error), - } - })) - .await? - .into_iter() - .flatten() - .collect(); - Ok(indexes_metadatas) - } - - fn uri(&self) -> &Uri { - self.storage.uri() - } - - async fn check_connectivity(&self) -> anyhow::Result<()> { - check_indexes_states_exist(self.storage.clone()).await + let metastore = self.clone(); + let indexes_metadata: Vec = try_join_all( + index_ids + .into_iter() + .map(|index_id| get_index_metadata(metastore.clone(), index_id)), + ) + .await? + .into_iter() + .flatten() + .collect(); + let response = ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata)?; + Ok(response) } // Shard API - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> MetastoreResult { let mut response = OpenShardsResponse { subresponses: Vec::with_capacity(request.subrequests.len()), }; @@ -675,7 +736,7 @@ impl Metastore for FileBackedMetastore { } async fn acquire_shards( - &self, + &mut self, request: AcquireShardsRequest, ) -> MetastoreResult { let mut response = AcquireShardsResponse { @@ -698,7 +759,7 @@ impl Metastore for FileBackedMetastore { } async fn close_shards( - &self, + &mut self, request: CloseShardsRequest, ) -> MetastoreResult { let mut response = CloseShardsResponse { @@ -727,7 +788,7 @@ impl Metastore for FileBackedMetastore { } async fn delete_shards( - &self, + &mut self, request: DeleteShardsRequest, ) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); @@ -751,7 +812,10 @@ impl Metastore for FileBackedMetastore { Ok(response) } - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); for subrequest in request.subrequests { @@ -768,12 +832,22 @@ impl Metastore for FileBackedMetastore { /// ------------------------------------------------------------------------------- /// Delete tasks - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - self.read(index_uid, |index| Ok(index.last_delete_opstamp())) - .await + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> MetastoreResult { + let last_delete_opstamp = self + .read(request.index_uid.into(), |index| { + Ok(index.last_delete_opstamp()) + }) + .await?; + Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) } - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { + async fn create_delete_task( + &mut self, + delete_query: DeleteQuery, + ) -> MetastoreResult { let index_uid: IndexUid = delete_query.index_uid.clone().into(); let delete_task = self .mutate(index_uid, |index| { @@ -785,36 +859,44 @@ impl Metastore for FileBackedMetastore { Ok(delete_task) } - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + self.mutate(index_uid, |index| { + let split_ids_str = request + .split_ids + .iter() + .map(|split_id| split_id.as_str()) + .collect::>(); index - .update_splits_delete_opstamp(split_ids, delete_opstamp) + .update_splits_delete_opstamp(&split_ids_str, request.delete_opstamp) .map(MutationOccurred::from) }) .await?; - Ok(()) + Ok(UpdateSplitsDeleteOpstampResponse {}) } async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + &mut self, + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + let delete_tasks = self - .read( - index_uid, - |index| Ok(index.list_delete_tasks(opstamp_start)), - ) + .read(index_uid, |index| { + Ok(index.list_delete_tasks(request.opstamp_start)) + }) .await??; - Ok(delete_tasks) + let response = ListDeleteTasksResponse { delete_tasks }; + Ok(response) } } +impl MetastoreServiceExt for FileBackedMetastore {} + async fn get_index_mutex( index_id: &str, index_state: &IndexState, @@ -836,6 +918,31 @@ async fn get_index_mutex( } } +async fn get_index_metadata( + mut metastore: FileBackedMetastore, + index_id: String, +) -> MetastoreResult> { + let request = IndexMetadataRequest::for_index_id(index_id); + let index_metadata_result = metastore + .index_metadata(request) + .await + .and_then(|response| response.deserialize_index_metadata()); + match index_metadata_result { + Ok(index_metadata) => Ok(Some(index_metadata)), + Err(MetastoreError::NotFound { .. }) => Ok(None), + Err(MetastoreError::Internal { message, cause }) => { + // Indexes can be in transient states `Creating` or `Deleting`. + // It is fine to ignore those errors. + if message.contains("transient state") { + Ok(None) + } else { + Err(MetastoreError::Internal { message, cause }) + } + } + Err(error) => Err(error), + } +} + /// Returns a [`RegexSet`] built from the following rules: /// - If the given pattern does not contain a `*` char, it matches the exact pattern. /// - If the given pattern contains one or more `*`, it matches the regex built from a regex where @@ -887,7 +994,7 @@ mod tests { use futures::executor::block_on; use quickwit_config::IndexConfig; use quickwit_proto::metastore::{DeleteQuery, MetastoreError}; - use quickwit_query::query_ast::qast_json_helper; + use quickwit_query::query_ast::qast_helper; use quickwit_storage::{MockStorage, RamStorage, Storage, StorageErrorKind}; use rand::Rng; use time::OffsetDateTime; @@ -899,7 +1006,14 @@ mod tests { }; use super::*; use crate::tests::test_suite::DefaultForTest; - use crate::{IndexMetadata, ListSplitsQuery, Metastore, SplitMetadata, SplitState}; + use crate::{IndexMetadata, ListSplitsQuery, SplitMetadata, SplitState}; + + #[tokio::test] + async fn test_metastore_connectivity_and_endpoints() { + let mut metastore = FileBackedMetastore::default_for_test().await; + metastore.check_connectivity().await.unwrap(); + assert!(metastore.endpoints()[0].protocol().is_ram()); + } #[tokio::test] async fn test_file_backed_metastore_connectivity_fails_if_states_file_does_not_exist() { @@ -917,7 +1031,7 @@ mod tests { assert!(path == Path::new("indexes_states.json")); block_on(ram_storage_clone.put(path, put_payload)) }); - let metastore = FileBackedMetastore::try_new(Arc::new(mock_storage), None) + let mut metastore = FileBackedMetastore::try_new(Arc::new(mock_storage), None) .await .unwrap(); @@ -927,23 +1041,31 @@ mod tests { #[tokio::test] async fn test_file_backed_metastore_index_exists() { let index_id = "test-index"; - let metastore = FileBackedMetastore::default_for_test().await; + let mut metastore = FileBackedMetastore::default_for_test().await; assert!(!metastore.index_exists(index_id).await.unwrap()); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); - let _index_uid = metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + metastore.create_index(create_index_request).await.unwrap(); assert!(metastore.index_exists(index_id).await.unwrap()); } #[tokio::test] async fn test_file_backed_metastore_get_index() { - let index_id = "test-index"; - let metastore = FileBackedMetastore::default_for_test().await; - let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); + let mut metastore = FileBackedMetastore::default_for_test().await; // Create index - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + let index_id = "test-index"; + let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-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(); // Open index and check its metadata let created_index = metastore.get_index(index_uid).await.unwrap(); @@ -954,11 +1076,13 @@ mod tests { ); // Check index is returned by list indexes. - let indexes = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + let indexes_metadata = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .unwrap() + .deserialize_indexes_metadata() .unwrap(); - assert_eq!(indexes.len(), 1); + assert_eq!(indexes_metadata.len(), 1); // Open a non-existent index. let metastore_error = metastore @@ -1002,13 +1126,20 @@ mod tests { .times(1) .returning(move |path| block_on(ram_storage.get_all(path))); mock_storage.expect_put().times(1).returning(|_uri, _| { - Err(StorageErrorKind::Io.with_error(anyhow::anyhow!( - "oops. perhaps there are some network problems" - ))) + Err(StorageErrorKind::Io + .with_error(anyhow::anyhow!("Oops. Some network problem maybe?"))) }); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + + let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); + 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 index_uid = IndexUid::new("test-index"); let split_id = "split-one"; let split_metadata = SplitMetadata { footer_offsets: 1000..2000, @@ -1019,35 +1150,35 @@ mod tests { create_timestamp: current_timestamp, ..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 index_config = IndexConfig::for_test(index_uid.index_id(), "ram:///indexes/test-index"); - - // create index - let index_uid = metastore.create_index(index_config).await.unwrap(); - - // stage split + // publish split fails + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + staged_split_ids: vec![split_id.to_string()], + ..Default::default() + }; metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) + .publish_splits(publish_splits_request) .await - .unwrap(); - - // publish split fails - let err = metastore - .publish_splits(index_uid.clone(), &[split_id], &[], None, None) - .await; - assert!(err.is_err()); + .unwrap_err(); - // empty let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); - let split = metastore.list_splits(query).await.unwrap(); - assert!(split.is_empty()); + 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(); + assert!(splits.is_empty()); - // not empty - let query = + let list_splits_query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let split = metastore.list_splits(query).await.unwrap(); - assert!(!split.is_empty()); + 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(); + assert!(!splits.is_empty()); } #[tokio::test] @@ -1090,15 +1221,21 @@ mod tests { } #[tokio::test] - async fn test_file_backed_metastore_wrt_directly_visible() -> MetastoreResult<()> { - let metastore = FileBackedMetastore::default_for_test().await; + async fn test_file_backed_metastore_write_directly_visible() -> MetastoreResult<()> { + let mut metastore = FileBackedMetastore::default_for_test().await; + let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); - let index_uid = metastore.create_index(index_config).await?; + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let create_index_response = metastore.create_index(create_index_request).await.unwrap(); + let index_uid: IndexUid = create_index_response.index_uid.into(); + + let list_splits_response = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + assert!(splits.is_empty()); - assert!(metastore - .list_all_splits(index_uid.clone()) - .await? - .is_empty()); let split_metadata = SplitMetadata { footer_offsets: 1000..2000, split_id: "split1".to_string(), @@ -1107,38 +1244,53 @@ mod tests { time_range: Some(0..=99), ..Default::default() }; - assert!(metastore - .list_all_splits(index_uid.clone()) - .await? - .is_empty()); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await?; - assert_eq!(metastore.list_all_splits(index_uid.clone()).await?.len(), 1); + let stage_splits_request = + StageSplitsRequest::try_from_split_metadata(index_uid.clone(), split_metadata).unwrap(); + metastore.stage_splits(stage_splits_request).await?; + + let list_splits_response = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + assert_eq!(splits.len(), 1); Ok(()) } #[tokio::test] - async fn test_file_backed_metastore_polling() -> quickwit_proto::metastore::MetastoreResult<()> - { + async fn test_file_backed_metastore_polling() -> MetastoreResult<()> { let storage = Arc::new(RamStorage::default()); - let metastore_wrt = FileBackedMetastore::try_new(storage.clone(), None) + let mut metastore_write = FileBackedMetastore::try_new(storage.clone(), None) .await .unwrap(); let polling_interval = Duration::from_millis(20); - let metastore_read = FileBackedMetastore::try_new(storage.clone(), Some(polling_interval)) + let mut metastore_read = FileBackedMetastore::try_new(storage, Some(polling_interval)) .await .unwrap(); - let index_id = "test-index"; - let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); - let index_uid = metastore_wrt.create_index(index_config).await?; + let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let create_index_response = metastore_write + .create_index(create_index_request) + .await + .unwrap(); + let index_uid: IndexUid = create_index_response.index_uid.into(); + + let list_splits_response = metastore_write + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + assert!(splits.is_empty()); + + let list_splits_response = metastore_read + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + assert!(splits.is_empty()); - assert!(metastore_wrt - .list_all_splits(index_uid.clone()) - .await? - .is_empty()); let split_metadata = SplitMetadata { footer_offsets: 1000..2000, split_id: "split1".to_string(), @@ -1147,24 +1299,27 @@ mod tests { time_range: Some(0..=99), ..Default::default() }; - assert!(metastore_read - .list_all_splits(index_uid.clone()) - .await? - .is_empty()); - metastore_wrt - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await?; - assert!(metastore_read - .list_all_splits(index_uid.clone()) - .await? - .is_empty()); + let stage_splits_request = + 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 + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + assert!(splits.is_empty()); + for _ in 0..10 { tokio::time::sleep(polling_interval).await; - if !metastore_read - .list_all_splits(index_uid.clone()) - .await? - .is_empty() - { + + let list_splits_response = metastore_read + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) + .await + .unwrap(); + let splits = list_splits_response.deserialize_splits().unwrap(); + + if !splits.is_empty() { return Ok(()); } } @@ -1173,20 +1328,19 @@ mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 3)] async fn test_file_backed_metastore_race_condition() { - let metastore = Arc::new(FileBackedMetastore::default_for_test().await); - let index_uid = IndexUid::new("test-index"); + let mut metastore = FileBackedMetastore::default_for_test().await; - let index_config = IndexConfig::for_test(index_uid.index_id(), "ram:///indexes/test-index"); - - // Create index - let index_uid = metastore.create_index(index_config).await.unwrap(); + let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let create_index_response = metastore.create_index(create_index_request).await.unwrap(); + let index_uid: IndexUid = create_index_response.index_uid.into(); // Stage splits in multiple threads let mut handles = Vec::new(); let mut random_generator = rand::thread_rng(); for i in 1..=20 { let sleep_duration = Duration::from_millis(random_generator.gen_range(0..=200)); - let metastore = metastore.clone(); + let mut metastore = metastore.clone(); let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); let handle = tokio::spawn({ let index_uid = index_uid.clone(); @@ -1201,17 +1355,24 @@ mod tests { ..Default::default() }; // stage split - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); + let stage_splits_request = StageSplitsRequest::try_from_split_metadata( + index_uid.clone(), + split_metadata, + ) + .unwrap(); + metastore.stage_splits(stage_splits_request).await.unwrap(); tokio::time::sleep(sleep_duration).await; // publish split let split_id = format!("split-{i}"); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + staged_split_ids: vec![split_id.to_string()], + ..Default::default() + }; metastore - .publish_splits(index_uid.clone(), &[&split_id], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap(); } @@ -1221,42 +1382,55 @@ mod tests { futures::future::try_join_all(handles).await.unwrap(); - let query = + let list_splits_query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); - let splits = metastore.list_splits(query).await.unwrap(); + 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(); // Make sure that all 20 splits are in `Published` state. assert_eq!(splits.len(), 20); } #[tokio::test(flavor = "multi_thread", worker_threads = 3)] - async fn test_file_backed_metastore_list_indexes_metadata_race_condition() { - let metastore = Arc::new(FileBackedMetastore::default_for_test().await); + async fn test_file_backed_metastore_list_indexes_race_condition() { + let mut metastore = FileBackedMetastore::default_for_test().await; let mut index_uids = Vec::new(); for idx in 0..10 { let index_uid = IndexUid::new(format!("test-index-{idx}")); let index_config = IndexConfig::for_test(index_uid.index_id(), "ram:///indexes/test-index"); - let index_uid = metastore.create_index(index_config).await.unwrap(); + 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(); index_uids.push(index_uid); } // Delete indexes + call to list_indexes_metadata. let mut handles = Vec::new(); for index_uid in index_uids { + let delete_request = DeleteIndexRequest { + index_uid: index_uid.to_string(), + }; { - let metastore = metastore.clone(); + let mut metastore = metastore.clone(); let handle = tokio::spawn(async move { metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await .unwrap(); }); handles.push(handle); } { - let metastore = metastore.clone(); + let mut metastore = metastore.clone(); let handle = tokio::spawn(async move { - metastore.delete_index(index_uid).await.unwrap(); + metastore.delete_index(delete_request).await.unwrap(); }); handles.push(handle); } @@ -1282,19 +1456,22 @@ mod tests { .times(1) .returning(move |path, _| { assert!(path == Path::new("indexes_states.json")); - Err(StorageErrorKind::Io.with_error(anyhow::anyhow!( - "oops. perhaps there are some network problems" - ))) + Err(StorageErrorKind::Io + .with_error(anyhow::anyhow!("Oops. Some network problem maybe?"))) }); mock_storage .expect_get_all() .times(1) .returning(move |path| block_on(ram_storage.get_all(path))); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index. - let metastore_error = metastore.create_index(index_config).await.unwrap_err(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let metastore_error = metastore + .create_index(create_index_request) + .await + .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Try fetch the not created index. let created_index_error = metastore @@ -1329,19 +1506,22 @@ mod tests { if path == Path::new("indexes_states.json") { return block_on(ram_storage_clone.put(path, put_payload)); } - Err(StorageErrorKind::Io.with_error(anyhow::anyhow!( - "oops. perhaps there are some network problems" - ))) + Err(StorageErrorKind::Io + .with_error(anyhow::anyhow!("Oops. Some network problem maybe?"))) }); mock_storage .expect_get_all() .times(1) .returning(move |path| block_on(ram_storage.get_all(path))); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index - let metastore_error = metastore.create_index(index_config).await.unwrap_err(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let metastore_error = metastore + .create_index(create_index_request) + .await + .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Let's fetch the index, we expect an internal error as the index state is in `Creating` // state. @@ -1360,7 +1540,10 @@ mod tests { IndexState::Creating )); // Let's delete the index to clean states. - let deleted_index_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest { + index_uid: index_uid.to_string(), + }; + let deleted_index_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!( deleted_index_error, MetastoreError::NotFound { .. } @@ -1405,11 +1588,14 @@ mod tests { } return block_on(ram_storage_clone.put(path, put_payload)); }); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index - let metastore_error = metastore.create_index(index_config).await.unwrap_err(); + let metastore_error = metastore + .create_index(CreateIndexRequest::try_from_index_config(index_config).unwrap()) + .await + .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Let's fetch the index, we expect an internal error as the index state is in `Creating` // state. @@ -1443,9 +1629,8 @@ mod tests { mock_storage // remove this if we end up changing the semantics of create. .expect_delete() .returning(|_| { - Err(StorageErrorKind::Io.with_error(anyhow::anyhow!( - "oops. perhaps there are some network problems" - ))) + Err(StorageErrorKind::Io + .with_error(anyhow::anyhow!("Oops. Some network problem maybe?"))) }); mock_storage .expect_put() @@ -1453,10 +1638,13 @@ mod tests { .returning(move |path, put_payload| { return block_on(ram_storage_clone.put(path, put_payload)); }); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index - let metastore_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest { + index_uid: index_uid.to_string(), + }; + let metastore_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Let's fetch the index, we expect an internal error as the index state is in `Deleting` // state. @@ -1503,10 +1691,13 @@ mod tests { } return block_on(ram_storage_clone.put(path, put_payload)); }); - let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index - let metastore_error = metastore.delete_index(index_uid.clone()).await.unwrap_err(); + let delete_request = DeleteIndexRequest { + index_uid: index_uid.to_string(), + }; + let metastore_error = metastore.delete_index(delete_request).await.unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Let's fetch the index, we expect an internal error as the index state is in `Deleting` // state. @@ -1564,14 +1755,16 @@ mod tests { .await?; // Fetch alive indexes metadatas. - let metastore = FileBackedMetastore::try_new(ram_storage.clone(), None) + let mut metastore = FileBackedMetastore::try_new(ram_storage.clone(), None) .await .unwrap(); - let indexes_metadatas = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + let indexes_metadata = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .unwrap() + .deserialize_indexes_metadata() .unwrap(); - assert_eq!(indexes_metadatas.len(), 1); + assert_eq!(indexes_metadata.len(), 1); // Fetch the index metadata not registered in indexes states json. metastore @@ -1581,26 +1774,31 @@ mod tests { // Now list indexes return 2 indexes metadatas as the metastore is now aware of // 2 alive indexes. - let indexes_metadatas = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + let indexes_metadata = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .unwrap() + .deserialize_indexes_metadata() .unwrap(); - assert_eq!(indexes_metadatas.len(), 2); + assert_eq!(indexes_metadata.len(), 2); // Let's delete indexes. - metastore - .delete_index(index_uid_alive.clone()) - .await - .unwrap(); - metastore - .delete_index(index_uid_unregistered.clone()) - .await - .unwrap(); - let no_more_indexes = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + let delete_request = DeleteIndexRequest { + index_uid: index_uid_alive.to_string(), + }; + metastore.delete_index(delete_request).await.unwrap(); + + let delete_request = DeleteIndexRequest { + index_uid: index_uid_unregistered.to_string(), + }; + metastore.delete_index(delete_request).await.unwrap(); + let indexes_metadata = metastore + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .unwrap() + .deserialize_indexes_metadata() .unwrap(); - assert!(no_more_indexes.is_empty()); + assert!(indexes_metadata.is_empty()); Ok(()) } @@ -1608,7 +1806,7 @@ mod tests { #[tokio::test] async fn test_monotically_increasing_stamps_by_index() { let storage = RamStorage::default(); - let metastore = FileBackedMetastore::try_new(Arc::new(storage.clone()), None) + let mut metastore = FileBackedMetastore::try_new(Arc::new(storage.clone()), None) .await .unwrap(); let index_id = "test-index-increasing-stamps-by-index"; @@ -1616,12 +1814,15 @@ mod tests { index_id, "ram:///indexes/test-index-increasing-stamps-by-index", ); - let index_uid = metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let create_index_response = metastore.create_index(create_index_request).await.unwrap(); + let index_uid = create_index_response.index_uid; + let delete_query = DeleteQuery { start_timestamp: None, end_timestamp: None, - index_uid: index_uid.to_string(), - query_ast: qast_json_helper("harry potter", &["body"]), + index_uid, + query_ast: serde_json::to_string(&qast_helper("harry potter", &["body"])).unwrap(), }; let delete_task_1 = metastore @@ -1636,7 +1837,7 @@ mod tests { assert_eq!(delete_task_2.opstamp, 2); // Create metastore with data already in the storage. - let new_metastore = FileBackedMetastore::try_new(Arc::new(storage), None) + let mut new_metastore = FileBackedMetastore::try_new(Arc::new(storage), None) .await .unwrap(); let delete_task_3 = new_metastore @@ -1651,19 +1852,20 @@ mod tests { index_id_2, "ram:///indexes/test-index-increasing-stamps-by-index-2", ); - let index_uid = metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let create_index_response = metastore.create_index(create_index_request).await.unwrap(); + let index_uid = create_index_response.index_uid; + let delete_query = DeleteQuery { start_timestamp: None, end_timestamp: None, - index_uid: index_uid.to_string(), - query_ast: qast_json_helper("harry potter", &["body"]), + index_uid, + query_ast: serde_json::to_string(&qast_helper("harry potter", &["body"])).unwrap(), }; - let delete_task_4 = metastore - .create_delete_task(delete_query.clone()) - .await - .unwrap(); + let delete_task_4 = metastore.create_delete_task(delete_query).await.unwrap(); assert_eq!(delete_task_4.opstamp, 1); } + #[test] fn test_build_regexes_from_pattern() { assert_eq!(build_regex_exprs_from_pattern("*").unwrap(), r"^.*$",); diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs deleted file mode 100644 index 7cc0c3fe2e5..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs +++ /dev/null @@ -1,517 +0,0 @@ -// 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::sync::Arc; - -use async_trait::async_trait; -use itertools::Itertools; -use quickwit_config::IndexConfig; -use quickwit_proto::metastore::{ - serde_utils as metastore_serde_utils, AcquireShardsRequest, AcquireShardsResponse, - AddSourceRequest, CloseShardsRequest, CloseShardsResponse, CreateIndexRequest, - CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, - DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, - IndexMetadataRequest, IndexMetadataResponse, LastDeleteOpstampRequest, - LastDeleteOpstampResponse, ListAllSplitsRequest, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexesMetadatasRequest, ListIndexesMetadatasResponse, - ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, - OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, - StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, - UpdateSplitsDeleteOpstampResponse, -}; -use quickwit_proto::tonic::{Request, Response, Status}; -use quickwit_proto::{set_parent_span_from_request_metadata, tonic}; -use tracing::instrument; - -use crate::{ListSplitsQuery, Metastore}; - -#[allow(missing_docs)] -#[derive(Clone)] -pub struct GrpcMetastoreAdapter(Arc); - -impl From> for GrpcMetastoreAdapter { - fn from(metastore: Arc) -> Self { - Self(metastore) - } -} - -#[async_trait] -impl MetastoreService for GrpcMetastoreAdapter { - #[instrument(skip(self, request))] - async fn create_index( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let create_index_request = request.into_inner(); - let index_config = serde_json::from_str::( - &create_index_request.index_config_json, - ) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "IndexConfig".to_string(), - message: error.to_string(), - })?; - let create_index_reply = - self.0 - .create_index(index_config) - .await - .map(|index_uid| CreateIndexResponse { - index_uid: index_uid.to_string(), - })?; - Ok(tonic::Response::new(create_index_reply)) - } - - #[instrument(skip(self, request))] - async fn index_metadata( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let index_metadata_request = request.into_inner(); - let index_metadata = self - .0 - .index_metadata(&index_metadata_request.index_id) - .await?; - let index_metadata_reply = serde_json::to_string(&index_metadata) - .map(|index_metadata_serialized_json| IndexMetadataResponse { - index_metadata_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "IndexMetadata".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(index_metadata_reply)) - } - - #[instrument(skip(self, request))] - async fn list_indexes_metadatas( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let query = serde_json::from_str(&request.into_inner().query_json).map_err(|error| { - MetastoreError::JsonSerializeError { - struct_name: "ListIndexesQuery".to_string(), - message: error.to_string(), - } - })?; - let indexes_metadatas = self.0.list_indexes_metadatas(query).await?; - let list_indexes_metadatas_reply = serde_json::to_string(&indexes_metadatas) - .map( - |indexes_metadatas_serialized_json| ListIndexesMetadatasResponse { - indexes_metadatas_serialized_json, - }, - ) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_indexes_metadatas_reply)) - } - - #[instrument(skip(self, request))] - async fn delete_index( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let delete_request = request.into_inner(); - let delete_reply = self - .0 - .delete_index(delete_request.index_uid.into()) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(delete_reply)) - } - - #[instrument(skip(self, request))] - async fn list_all_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let list_all_splits_request = request.into_inner(); - let splits = self - .0 - .list_all_splits(list_all_splits_request.index_uid.into()) - .await?; - let list_all_splits_reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_all_splits_reply)) - } - - #[instrument(skip(self, request))] - async fn list_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let list_splits_request = request.into_inner(); - let query: ListSplitsQuery = serde_json::from_str(&list_splits_request.query_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "ListSplitsQuery".to_string(), - message: error.to_string(), - })?; - - let splits = self.0.list_splits(query).await?; - let list_splits_reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(list_splits_reply)) - } - - #[instrument(skip(self, request))] - async fn stage_splits( - &self, - request: Request, - ) -> Result, Status> { - set_parent_span_from_request_metadata(request.metadata()); - let stage_split_request = request.into_inner(); - let split_metadata_list = - serde_json::from_str(&stage_split_request.split_metadata_list_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - self.0 - .stage_splits(stage_split_request.index_uid.into(), split_metadata_list) - .await?; - Ok(tonic::Response::new(EmptyResponse {})) - } - - #[instrument(skip(self, request))] - async fn publish_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let publish_request = request.into_inner(); - let split_ids = publish_request - .staged_split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let replaced_split_ids = publish_request - .replaced_split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let checkpoint_delta_opt = publish_request - .index_checkpoint_delta_json_opt - .as_deref() - .map(metastore_serde_utils::from_json_str) - .transpose()?; - let publish_splits_reply = self - .0 - .publish_splits( - publish_request.index_uid.into(), - &split_ids, - &replaced_split_ids, - checkpoint_delta_opt, - publish_request.publish_token_opt, - ) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(publish_splits_reply)) - } - - #[instrument(skip(self, request))] - async fn mark_splits_for_deletion( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let mark_splits_for_deletion_request = request.into_inner(); - let split_ids = mark_splits_for_deletion_request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let mark_splits_for_deletion_reply = self - .0 - .mark_splits_for_deletion( - mark_splits_for_deletion_request.index_uid.into(), - &split_ids, - ) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(mark_splits_for_deletion_reply)) - } - - #[instrument(skip(self, request))] - async fn delete_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let delete_splits_request = request.into_inner(); - let split_ids = delete_splits_request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let delete_splits_reply = self - .0 - .delete_splits(delete_splits_request.index_uid.into(), &split_ids) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(delete_splits_reply)) - } - - #[instrument(skip(self, request))] - async fn add_source( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let add_source_request = request.into_inner(); - let source_config = - serde_json::from_str(&add_source_request.source_config_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "SourceConfig".to_string(), - message: error.to_string(), - } - })?; - let add_source_reply = self - .0 - .add_source(add_source_request.index_uid.into(), source_config) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(add_source_reply)) - } - - #[instrument(skip(self, request))] - async fn toggle_source( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let toggle_source_request = request.into_inner(); - let toggle_source_reply = self - .0 - .toggle_source( - toggle_source_request.index_uid.into(), - &toggle_source_request.source_id, - toggle_source_request.enable, - ) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(toggle_source_reply)) - } - - #[instrument(skip(self, request))] - async fn delete_source( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let delete_source_request = request.into_inner(); - let delete_source_reply = self - .0 - .delete_source( - delete_source_request.index_uid.into(), - &delete_source_request.source_id, - ) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(delete_source_reply)) - } - - #[instrument(skip(self, request))] - async fn reset_source_checkpoint( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let reply = self - .0 - .reset_source_checkpoint(request.index_uid.into(), &request.source_id) - .await - .map(|_| EmptyResponse {})?; - Ok(tonic::Response::new(reply)) - } - - #[instrument(skip(self, request))] - async fn last_delete_opstamp( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let last_delete_opstamp = self.0.last_delete_opstamp(request.index_uid.into()).await?; - let last_opstamp_reply = LastDeleteOpstampResponse { - last_delete_opstamp, - }; - Ok(tonic::Response::new(last_opstamp_reply)) - } - - #[instrument(skip(self, request))] - async fn create_delete_task( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let delete_task = self.0.create_delete_task(request).await?; - Ok(tonic::Response::new(delete_task)) - } - - #[instrument(skip(self, request))] - async fn update_splits_delete_opstamp( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let split_ids = request - .split_ids - .iter() - .map(|split_id| split_id.as_str()) - .collect_vec(); - let reply = self - .0 - .update_splits_delete_opstamp( - request.index_uid.into(), - &split_ids, - request.delete_opstamp, - ) - .await - .map(|_| UpdateSplitsDeleteOpstampResponse {})?; - Ok(tonic::Response::new(reply)) - } - - #[instrument(skip(self, request))] - async fn list_delete_tasks( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let delete_tasks = self - .0 - .list_delete_tasks(request.index_uid.into(), request.opstamp_start) - .await? - .into_iter() - .map(DeleteTask::from) - .collect_vec(); - let reply = ListDeleteTasksResponse { delete_tasks }; - Ok(tonic::Response::new(reply)) - } - - #[instrument(skip(self, request))] - async fn list_stale_splits( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let splits = self - .0 - .list_stale_splits( - request.index_uid.into(), - request.delete_opstamp, - request.num_splits as usize, - ) - .await?; - let reply = serde_json::to_string(&splits) - .map(|splits_serialized_json| ListSplitsResponse { - splits_serialized_json, - }) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(tonic::Response::new(reply)) - } - - // Shard API: - // - `open_shards` - // - `acquire_shards` - // - `close_shards` - // - `list_shards` - // - `delete_shards` - - #[instrument(skip(self, request))] - async fn open_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let response = self.0.open_shards(request).await?; - Ok(tonic::Response::new(response)) - } - - #[instrument(skip(self, request))] - async fn acquire_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let response = self.0.acquire_shards(request).await?; - Ok(tonic::Response::new(response)) - } - - #[instrument(skip(self, request))] - async fn close_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let response = self.0.close_shards(request).await?; - Ok(tonic::Response::new(response)) - } - - #[instrument(skip(self, request))] - async fn list_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let response = self.0.list_shards(request).await?; - Ok(tonic::Response::new(response)) - } - - #[instrument(skip(self, request))] - async fn delete_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - set_parent_span_from_request_metadata(request.metadata()); - let request = request.into_inner(); - let response = self.0.delete_shards(request).await?; - Ok(tonic::Response::new(response)) - } -} diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs deleted file mode 100644 index 3028bf48875..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs +++ /dev/null @@ -1,670 +0,0 @@ -// 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 . - -mod grpc_adapter; - -use std::error::Error; -use std::fmt; -use std::net::SocketAddr; - -use anyhow::bail; -use async_trait::async_trait; -pub use grpc_adapter::GrpcMetastoreAdapter; -use itertools::Itertools; -use quickwit_common::tower::BalanceChannel; -use quickwit_common::uri::Uri as QuickwitUri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{ - serde_utils as metastore_serde_utils, AcquireShardsRequest, AcquireShardsResponse, - AddSourceRequest, CloseShardsRequest, CloseShardsResponse, CreateIndexRequest, - DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, - DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, IndexMetadataRequest, - LastDeleteOpstampRequest, ListAllSplitsRequest, ListDeleteTasksRequest, - ListIndexesMetadatasRequest, ListShardsRequest, ListShardsResponse, ListSplitsRequest, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, - MetastoreServiceClient, OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, - ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateSplitsDeleteOpstampRequest, -}; -use quickwit_proto::tonic::codegen::InterceptedService; -use quickwit_proto::tonic::Status; -use quickwit_proto::{IndexUid, PublishToken, SpanContextInterceptor}; -use tower::timeout::error::Elapsed; - -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata}; - -// URI describing in a generic way the metastore services resource present in the cluster (= -// discovered by Quickwit gossip). This value is used to build the URI of `MetastoreGrpcClient` and -// is only useful for debugging. -const GRPC_METASTORE_BASE_URI: &str = "grpc://metastore.service.cluster"; - -type Transport = InterceptedService, SpanContextInterceptor>; -type MetastoreGrpcClientImpl = MetastoreServiceClient; - -/// The [`MetastoreGrpcClient`] sends gRPC requests to cluster members running a [`Metastore`] -/// service, those nodes will execute the queries on the metastore. -/// The [`MetastoreGrpcClient`] use tonic load balancer to balance requests between nodes and -/// listen to cluster live nodes changes to keep updated the list of available nodes. -#[derive(Clone)] -pub struct MetastoreGrpcClient { - underlying: MetastoreGrpcClientImpl, - balance_channel: BalanceChannel, - // URI used to describe the metastore resource of form - // `GRPC_METASTORE_BASE_URI:{grpc_advertise_port}`. This value is only useful for - // debugging. - uri: QuickwitUri, -} - -impl fmt::Debug for MetastoreGrpcClient { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("GrpcMetastore") - .field("uri", &self.uri) - .finish() - } -} - -impl MetastoreGrpcClient { - /// Create a [`MetastoreGrpcClient`] that sends gRPC requests to nodes running - /// [`Metastore`] service. It listens to cluster members changes to update the - /// nodes. - /// `grpc_advertise_port` is used only for building the `uri`. - pub async fn from_balance_channel( - balance_channel: BalanceChannel, - ) -> anyhow::Result { - let underlying = MetastoreServiceClient::with_interceptor( - balance_channel.clone(), - SpanContextInterceptor, - ); - let uri = QuickwitUri::from_well_formed(GRPC_METASTORE_BASE_URI); - Ok(Self { - underlying, - balance_channel, - uri, - }) - } - - /// Creates a [`MetastoreGrpcClient`] from a duplex stream client for testing purpose. - #[cfg(any(test, feature = "testsuite"))] - pub async fn from_duplex_stream(client: tokio::io::DuplexStream) -> anyhow::Result { - use http::Uri; - use quickwit_proto::tonic::transport::Endpoint; - - 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?; - let dummy_addr = "127.0.0.1:1234".parse::()?; - let balance_channel = BalanceChannel::from_channel(dummy_addr, channel); - let underlying = MetastoreServiceClient::with_interceptor( - balance_channel.clone(), - SpanContextInterceptor, - ); - Ok(Self { - underlying, - balance_channel, - uri: QuickwitUri::from_well_formed(GRPC_METASTORE_BASE_URI), - }) - } -} - -#[async_trait] -impl Metastore for MetastoreGrpcClient { - async fn check_connectivity(&self) -> anyhow::Result<()> { - if self.balance_channel.num_connections() == 0 { - bail!("the metastore service is unavailable"); - } - Ok(()) - } - - fn uri(&self) -> &QuickwitUri { - &self.uri - } - - /// Creates an index. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_config_json = serde_json::to_string(&index_config).map_err(|error| { - MetastoreError::JsonSerializeError { - struct_name: "IndexConfig".to_string(), - message: error.to_string(), - } - })?; - let request = CreateIndexRequest { index_config_json }; - let inner_response = self - .underlying - .clone() - .create_index(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))? - .into_inner(); - let index_uid = inner_response.index_uid.into(); - Ok(index_uid) - } - - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { - let query_json = serde_json::to_string(&query).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "ListIndexesQuery".to_string(), - message: error.to_string(), - } - })?; - let response = self - .underlying - .clone() - .list_indexes_metadatas(ListIndexesMetadatasRequest { query_json }) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let indexes_metadatas = - serde_json::from_str(&response.into_inner().indexes_metadatas_serialized_json) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - Ok(indexes_metadatas) - } - - /// Returns the [`IndexMetadata`] for a given index. - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - let request = IndexMetadataRequest { - index_id: index_id.to_string(), - }; - let response = self - .underlying - .clone() - .index_metadata(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let index_metadata = serde_json::from_str( - &response.into_inner().index_metadata_serialized_json, - ) - .map_err(|error| MetastoreError::JsonDeserializeError { - struct_name: "IndexMetadata".to_string(), - message: error.to_string(), - })?; - Ok(index_metadata) - } - - /// Deletes an index. - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - let request = DeleteIndexRequest { - index_uid: index_uid.to_string(), - }; - self.underlying - .clone() - .delete_index(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Stages several splits. - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - let split_metadata_list_serialized_json = serde_json::to_string(&split_metadata_list) - .map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - })?; - let tonic_request = StageSplitsRequest { - index_uid: index_uid.to_string(), - split_metadata_list_serialized_json, - }; - self.underlying - .clone() - .stage_splits(tonic_request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Publishes a list of splits. - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - let staged_split_ids: Vec = staged_split_ids - .iter() - .map(|split| split.to_string()) - .collect(); - let replaced_split_ids: Vec = replaced_split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let index_checkpoint_delta_json_opt = checkpoint_delta_opt - .as_ref() - .map(metastore_serde_utils::to_json_str) - .transpose()?; - let request = PublishSplitsRequest { - index_uid: index_uid.into(), - staged_split_ids, - replaced_split_ids, - index_checkpoint_delta_json_opt, - publish_token_opt, - }; - self.underlying - .clone() - .publish_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Lists the splits. - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - let query_json = - serde_json::to_string(&query).map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "ListSplitsQuery".to_string(), - message: error.to_string(), - })?; - - let request = ListSplitsRequest { query_json }; - let response = self - .underlying - .clone() - .list_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) - } - - /// Lists all the splits without filtering. - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - let request = ListAllSplitsRequest { - index_uid: index_uid.into(), - }; - let response = self - .underlying - .clone() - .list_all_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) - } - - /// Marks a list of splits for deletion. - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = MarkSplitsForDeletionRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - }; - self.underlying - .clone() - .mark_splits_for_deletion(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Deletes a list of splits. - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = DeleteSplitsRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - }; - self.underlying - .clone() - .delete_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Adds a source to a given index. - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - let source_config_json = - serde_json::to_string(&source).map_err(|error| MetastoreError::JsonSerializeError { - struct_name: "SourceConfig".to_string(), - message: error.to_string(), - })?; - let request = AddSourceRequest { - index_uid: index_uid.into(), - source_config_json, - }; - self.underlying - .clone() - .add_source(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Toggles the source `enabled` field value. - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - let request = ToggleSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - enable, - }; - self.underlying - .clone() - .toggle_source(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Removes a source from a given index. - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - let request = DeleteSourceRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - }; - self.underlying - .clone() - .delete_source(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - /// Resets a source checkpoint. - async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - let request = ResetSourceCheckpointRequest { - index_uid: index_uid.into(), - source_id: source_id.to_string(), - }; - self.underlying - .clone() - .reset_source_checkpoint(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - let request = LastDeleteOpstampRequest { - index_uid: index_uid.into(), - }; - let response = self - .underlying - .clone() - .last_delete_opstamp(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.last_delete_opstamp) - } - - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { - let response = self - .underlying - .clone() - .create_delete_task(delete_query) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response) - } - - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let request = UpdateSplitsDeleteOpstampRequest { - index_uid: index_uid.into(), - split_ids: split_ids_vec, - delete_opstamp, - }; - self.underlying - .clone() - .update_splits_delete_opstamp(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(()) - } - - async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - let request = ListDeleteTasksRequest { - index_uid: index_uid.into(), - opstamp_start, - }; - let response = self - .underlying - .clone() - .list_delete_tasks(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let delete_tasks: Vec = response - .delete_tasks - .into_iter() - .map(DeleteTask::from) - .collect_vec(); - Ok(delete_tasks) - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - let request = ListStaleSplitsRequest { - index_uid: index_uid.into(), - delete_opstamp, - num_splits: num_splits as u64, - }; - let response = self - .underlying - .clone() - .list_stale_splits(request) - .await - .map(|tonic_response| tonic_response.into_inner()) - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - let splits: Vec = - serde_json::from_str(&response.splits_serialized_json).map_err(|error| { - MetastoreError::JsonDeserializeError { - struct_name: "Vec".to_string(), - message: error.to_string(), - } - })?; - Ok(splits) - } - - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { - let response = self - .underlying - .clone() - .open_shards(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.into_inner()) - } - - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult { - let response = self - .underlying - .clone() - .acquire_shards(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.into_inner()) - } - - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> MetastoreResult { - let response = self - .underlying - .clone() - .close_shards(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.into_inner()) - } - - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - let response = self - .underlying - .clone() - .list_shards(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.into_inner()) - } - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> MetastoreResult { - let response = self - .underlying - .clone() - .delete_shards(request) - .await - .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; - Ok(response.into_inner()) - } -} - -/// Parse tonic error and returns [`MetastoreError`]. -pub fn parse_grpc_error(grpc_error: &Status) -> MetastoreError { - // TODO: we want to process network related errors so that we - // return the right `MetastoreError`. We do it for the - // channel timeout error to help the user debug this kind of - // situation. - let elapsed_error_opt = grpc_error - .source() - .and_then(|error| error.downcast_ref::()); - - if elapsed_error_opt.is_some() { - return MetastoreError::Connection { - message: "gRPC request timeout triggered by the channel timeout. this can happens \ - when tonic channel has no registered endpoints" - .to_string(), - }; - } - - serde_json::from_str(grpc_error.message()).unwrap_or_else(|_| MetastoreError::Internal { - message: grpc_error.message().to_string(), - cause: "".to_string(), - }) -} - -#[cfg(test)] -#[async_trait] -impl crate::tests::test_suite::DefaultForTest for MetastoreGrpcClient { - async fn default_for_test() -> Self { - use std::sync::Arc; - - use quickwit_proto::metastore::MetastoreServiceServer; - use quickwit_proto::tonic::transport::Server; - use quickwit_storage::RamStorage; - - use crate::FileBackedMetastore; - - let metastore = FileBackedMetastore::try_new(Arc::new(RamStorage::default()), None) - .await - .unwrap(); - let (client, server) = tokio::io::duplex(1024); - let grpc_adapter = GrpcMetastoreAdapter::from(Arc::new(metastore) as Arc); - tokio::spawn(async move { - Server::builder() - .add_service(MetastoreServiceServer::new(grpc_adapter)) - .serve_with_incoming(futures::stream::iter(vec![Ok::<_, std::io::Error>(server)])) - .await - }); - MetastoreGrpcClient::from_duplex_stream(client) - .await - .unwrap() - } -} - -metastore_test_suite!(crate::MetastoreGrpcClient); diff --git a/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs b/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs deleted file mode 100644 index bdc5364b5b2..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/instrumented_metastore.rs +++ /dev/null @@ -1,390 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use std::fmt; - -use async_trait::async_trait; -use itertools::Itertools; -use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, ListShardsRequest, - ListShardsResponse, MetastoreResult, OpenShardsRequest, OpenShardsResponse, -}; -use quickwit_proto::{IndexUid, PublishToken}; - -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata}; - -macro_rules! instrument { - ($expr:expr, [$operation:ident, $($label:expr),*]) => { - let start = std::time::Instant::now(); - let labels = [stringify!($operation), $($label,)*]; - crate::metrics::METASTORE_METRICS.requests_total.with_label_values(labels).inc(); - let (res, is_error) = match $expr { - ok @ Ok(_) => { - (ok, "false") - }, - err @ Err(_) => { - crate::metrics::METASTORE_METRICS.request_errors_total.with_label_values(labels).inc(); - (err, "true") - }, - }; - let elapsed = start.elapsed(); - let labels = [stringify!($operation), $($label,)* is_error]; - crate::metrics::METASTORE_METRICS.request_duration_seconds.with_label_values(labels).observe(elapsed.as_secs_f64()); - - if elapsed.as_secs() > 1 { - let index_id = if labels.len() > 2 { - labels[1] - } else { - "" - }; - tracing::warn!( - operation=stringify!($operation), - duration_millis=elapsed.as_millis(), - index_id=index_id, - "Slow metastore operation" - ); - } - return res; - }; -} - -pub(crate) struct InstrumentedMetastore { - underlying: Box, -} - -impl InstrumentedMetastore { - pub fn new(metastore: Box) -> Self { - Self { - underlying: metastore, - } - } -} - -impl fmt::Debug for InstrumentedMetastore { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("InstrumentedMetastore").finish() - } -} - -#[async_trait] -impl Metastore for InstrumentedMetastore { - fn uri(&self) -> &Uri { - self.underlying.uri() - } - - async fn check_connectivity(&self) -> anyhow::Result<()> { - self.underlying.check_connectivity().await - } - - // Index API - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - let index_id = index_config.index_id.clone(); - instrument!( - self.underlying.create_index(index_config).await, - [create_index, &index_id] - ); - } - - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - instrument!( - self.underlying.index_exists(index_id).await, - [index_exists, index_id] - ); - } - - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - instrument!( - self.underlying.index_metadata(index_id).await, - [index_metadata, index_id] - ); - } - - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { - instrument!( - self.underlying.list_indexes_metadatas(query).await, - [list_indexes_metadatas, ""] - ); - } - - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - instrument!( - self.underlying.delete_index(index_uid.clone()).await, - [delete_index, index_uid.index_id()] - ); - } - - // Split API - - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .stage_splits(index_uid.clone(), split_metadata_list) - .await, - [stage_splits, index_uid.index_id()] - ); - } - - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .publish_splits( - index_uid.clone(), - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - publish_token_opt, - ) - .await, - [publish_splits, index_uid.index_id()] - ); - } - - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - let index_ids = query - .index_uids - .iter() - .map(|index_uid| index_uid.index_id()) - .join(","); - instrument!( - self.underlying.list_splits(query).await, - [list_splits, &index_ids] - ); - } - - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - instrument!( - self.underlying.list_all_splits(index_uid.clone()).await, - [list_all_splits, index_uid.index_id()] - ); - } - - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .mark_splits_for_deletion(index_uid.clone(), split_ids) - .await, - [mark_splits_for_deletion, index_uid.index_id()] - ); - } - - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .delete_splits(index_uid.clone(), split_ids) - .await, - [delete_splits, index_uid.index_id()] - ); - } - - // Source API - - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - instrument!( - self.underlying.add_source(index_uid.clone(), source).await, - [add_source, index_uid.index_id()] - ); - } - - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .toggle_source(index_uid.clone(), source_id, enable) - .await, - [toggle_source, index_uid.index_id()] - ); - } - - async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .reset_source_checkpoint(index_uid.clone(), source_id) - .await, - [reset_source_checkpoint, index_uid.index_id()] - ); - } - - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - instrument!( - self.underlying - .delete_source(index_uid.clone(), source_id) - .await, - [delete_source, index_uid.index_id()] - ); - } - - // Delete task API - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { - let index_uid: IndexUid = delete_query.index_uid.clone().into(); - instrument!( - self.underlying.create_delete_task(delete_query).await, - [create_delete_task, index_uid.index_id()] - ); - } - - async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - instrument!( - self.underlying - .list_delete_tasks(index_uid.clone(), opstamp_start) - .await, - [list_delete_tasks, index_uid.index_id()] - ); - } - - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - instrument!( - self.underlying.last_delete_opstamp(index_uid.clone()).await, - [last_delete_opstamp, index_uid.index_id()] - ); - } - - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - instrument!( - self.underlying - .update_splits_delete_opstamp(index_uid.clone(), split_ids, delete_opstamp) - .await, - [update_splits_delete_opstamp, index_uid.index_id()] - ); - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - instrument!( - self.underlying - .list_stale_splits(index_uid.clone(), delete_opstamp, num_splits) - .await, - [list_stale_splits, index_uid.index_id()] - ); - } - - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { - instrument!( - self.underlying.open_shards(request).await, - [open_shards, ""] - ); - } - - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult { - instrument!( - self.underlying.acquire_shards(request).await, - [acquire_shards, ""] - ); - } - - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> MetastoreResult { - instrument!( - self.underlying.close_shards(request).await, - [close_shards, ""] - ); - } - - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - instrument!( - self.underlying.list_shards(request).await, - [list_shards, ""] - ); - } - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> MetastoreResult { - instrument!( - self.underlying.delete_shards(request).await, - [delete_shards, ""] - ); - } -} - -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use quickwit_storage::RamStorage; - - use super::*; - use crate::tests::test_suite::DefaultForTest; - use crate::FileBackedMetastore; - - #[async_trait] - impl DefaultForTest for InstrumentedMetastore { - async fn default_for_test() -> Self { - InstrumentedMetastore { - underlying: Box::new(FileBackedMetastore::for_test(Arc::new( - RamStorage::default(), - ))), - } - } - } - - metastore_test_suite!(crate::metastore::instrumented_metastore::InstrumentedMetastore); -} diff --git a/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs b/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs deleted file mode 100644 index 4485a0d5ea0..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/metastore_event_publisher.rs +++ /dev/null @@ -1,349 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use std::fmt; -use std::sync::Arc; - -use async_trait::async_trait; -use quickwit_common::pubsub::EventBroker; -use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::events::{ - AddSourceEvent, DeleteIndexEvent, DeleteSourceEvent, ToggleSourceEvent, -}; -use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, ListShardsRequest, - ListShardsResponse, MetastoreResult, OpenShardsRequest, OpenShardsResponse, -}; -use quickwit_proto::{IndexUid, PublishToken}; -use tracing::info; - -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata}; - -/// Wraps a metastore and dispatches events to subscribers. -pub struct MetastoreEventPublisher { - underlying: Arc, - event_broker: EventBroker, -} - -impl MetastoreEventPublisher { - /// Creates a new metastore publisher. - pub fn new(metastore: Arc, event_broker: EventBroker) -> Self { - Self { - underlying: metastore, - event_broker, - } - } -} - -impl fmt::Debug for MetastoreEventPublisher { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("MetastorePublisher") - .field("uri", self.underlying.uri()) - .finish() - } -} - -#[async_trait] -impl Metastore for MetastoreEventPublisher { - fn uri(&self) -> &Uri { - self.underlying.uri() - } - - async fn check_connectivity(&self) -> anyhow::Result<()> { - self.underlying.check_connectivity().await - } - - // Index API - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - self.underlying.create_index(index_config).await - } - - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - self.underlying.index_exists(index_id).await - } - - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - self.underlying.index_metadata(index_id).await - } - - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { - self.underlying.list_indexes_metadatas(query).await - } - - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - let event = DeleteIndexEvent { - index_uid: index_uid.clone(), - }; - self.underlying.delete_index(index_uid).await?; - self.event_broker.publish(event); - Ok(()) - } - - // Split API - - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - self.underlying - .stage_splits(index_uid, split_metadata_list) - .await - } - - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - self.underlying - .publish_splits( - index_uid, - split_ids, - replaced_split_ids, - checkpoint_delta_opt, - publish_token_opt, - ) - .await - } - - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - self.underlying.list_splits(query).await - } - - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - self.underlying.list_all_splits(index_uid).await - } - - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.underlying - .mark_splits_for_deletion(index_uid, split_ids) - .await - } - - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.underlying.delete_splits(index_uid, split_ids).await - } - - // Source API - - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - let event = AddSourceEvent { - index_uid: index_uid.clone(), - source_id: source.source_id.clone(), - source_type: source.source_type(), - }; - info!("add source {0}, {source:?}", index_uid.index_id()); - self.underlying.add_source(index_uid, source).await?; - self.event_broker.publish(event); - Ok(()) - } - - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - let event = ToggleSourceEvent { - index_uid: index_uid.clone(), - source_id: source_id.to_string(), - enabled: enable, - }; - self.underlying - .toggle_source(index_uid, source_id, enable) - .await?; - self.event_broker.publish(event); - Ok(()) - } - - async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - self.underlying - .reset_source_checkpoint(index_uid, source_id) - .await - } - - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - let event = DeleteSourceEvent { - index_uid: index_uid.clone(), - source_id: source_id.to_string(), - }; - self.underlying.delete_source(index_uid, source_id).await?; - self.event_broker.publish(event); - Ok(()) - } - - // Delete task API - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { - self.underlying.create_delete_task(delete_query).await - } - - async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - self.underlying - .list_delete_tasks(index_uid, opstamp_start) - .await - } - - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - self.underlying.last_delete_opstamp(index_uid).await - } - - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - self.underlying - .update_splits_delete_opstamp(index_uid, split_ids, delete_opstamp) - .await - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - self.underlying - .list_stale_splits(index_uid, delete_opstamp, num_splits) - .await - } - - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { - self.underlying.open_shards(request).await - } - - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult { - self.underlying.acquire_shards(request).await - } - - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> MetastoreResult { - self.underlying.close_shards(request).await - } - - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - self.underlying.list_shards(request).await - } - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> MetastoreResult { - self.underlying.delete_shards(request).await - } -} - -#[cfg(test)] -mod tests { - - use quickwit_common::pubsub::EventSubscriber; - use quickwit_config::SourceParams; - use quickwit_proto::metastore::SourceType; - - use super::*; - use crate::metastore_for_test; - use crate::tests::test_suite::DefaultForTest; - - #[async_trait] - impl DefaultForTest for MetastoreEventPublisher { - async fn default_for_test() -> Self { - MetastoreEventPublisher { - underlying: metastore_for_test(), - event_broker: EventBroker::default(), - } - } - } - - metastore_test_suite!(crate::metastore::metastore_event_publisher::MetastoreEventPublisher); - - #[derive(Debug, Clone)] - struct TxSubscriber(tokio::sync::mpsc::Sender); - - #[async_trait] - impl EventSubscriber for TxSubscriber { - async fn handle_event(&mut self, event: AddSourceEvent) { - let _ = self.0.send(event).await; - } - } - - #[tokio::test] - async fn test_metastore_event_publisher() { - let metastore = MetastoreEventPublisher::default_for_test().await; - - let (tx, mut rx) = tokio::sync::mpsc::channel(1); - let subscription = metastore.event_broker.subscribe(TxSubscriber(tx)); - - let index_uid = IndexUid::new("test-index"); - let index_uri = "ram:///indexes/test-index"; - let source_id = "test-source"; - let source_config = SourceConfig::for_test(source_id, SourceParams::void()); - - let index_uid = metastore - .create_index(IndexConfig::for_test(index_uid.index_id(), index_uri)) - .await - .unwrap(); - - metastore - .add_source(index_uid.clone(), source_config) - .await - .unwrap(); - - assert_eq!( - rx.recv().await.unwrap(), - AddSourceEvent { - index_uid: index_uid.clone(), - source_id: source_id.to_string(), - source_type: SourceType::Void, - } - ); - subscription.cancel(); - } -} diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 809b96e73d5..b22ddd653cf 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -17,336 +17,63 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -pub mod control_plane_metastore; pub mod file_backed_metastore; -pub mod grpc_metastore; pub(crate) mod index_metadata; -mod instrumented_metastore; -pub mod metastore_event_publisher; #[cfg(feature = "postgres")] pub mod postgresql_metastore; #[cfg(feature = "postgres")] mod postgresql_model; -pub mod retrying_metastore; -use std::fmt; +pub mod control_plane_metastore; + use std::ops::{Bound, RangeInclusive}; use async_trait::async_trait; pub use index_metadata::IndexMetadata; -use quickwit_common::uri::Uri; +use itertools::Itertools; +use once_cell::sync::Lazy; +use quickwit_common::tower::PrometheusMetricsLayer; use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_doc_mapper::tag_pruning::TagFilterAst; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, - ListShardsRequest, ListShardsResponse, MetastoreError, MetastoreResult, OpenShardsRequest, - OpenShardsResponse, + serde_utils, AddSourceRequest, CreateIndexRequest, DeleteTask, IndexMetadataRequest, + IndexMetadataResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListSplitsRequest, ListSplitsResponse, MetastoreError, MetastoreResult, MetastoreService, + MetastoreServiceClient, PublishSplitsRequest, StageSplitsRequest, }; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::{IndexUid, SplitId}; use time::OffsetDateTime; use crate::checkpoint::IndexCheckpointDelta; use crate::{Split, SplitMetadata, SplitState}; -/// Metastore meant to manage Quickwit's indexes, their splits and delete tasks. -/// -/// I. Index and splits management. -/// -/// Quickwit needs a way to ensure that we can cleanup unused files, -/// and this process needs to be resilient to any fail-stop failures. -/// We rely on atomically transitioning the status of splits. -/// -/// The split state goes through the following life cycle: -/// 1. `Staged` -/// - Start uploading the split files. -/// 2. `Published` -/// - Uploading the split files is complete and the split is searchable. -/// 3. `MarkedForDeletion` -/// - Mark the split for deletion. -/// -/// If a split has a file in the storage, it MUST be registered in the metastore, -/// and its state can be as follows: -/// - `Staged`: The split is almost ready. Some of its files may have been uploaded in the storage. -/// - `Published`: The split is ready and published. -/// - `MarkedForDeletion`: The split is marked for deletion. -/// -/// Before creating any file, we need to stage the split. If there is a failure, upon recovery, we -/// schedule for deletion all the staged splits. A client may not necessarily remove files from -/// storage right after marking it for deletion. A CLI client may delete files right away, but a -/// more serious deployment should probably only delete those files after a grace period so that the -/// running search queries can complete. -/// -/// II. Delete tasks management. -/// -/// A delete task is defined on a given index and by a search query. It can be -/// applied to all the splits of the index. -/// -/// Quickwit needs a way to track that a delete task has been applied to a split. This is ensured -/// by two mechanisms: -/// - On creation of a delete task, we give to the task a monotically increasing opstamp (uniqueness -/// and monotonically increasing must be true at the index level). -/// - When a delete task is executed on a split, that is when the documents matched by the search -/// query are removed from the splits, we update the split's `delete_opstamp` to the value of the -/// task's opstamp. This marks the split as "up-to-date" regarding this delete task. If new delete -/// tasks are added, we will know that we need to run these delete tasks on the splits as its -/// `delete_optstamp` will be inferior to the `opstamp` of the new tasks. -/// -/// For splits created after a given delete task, Quickwit's indexing ensures that these splits -/// are created with a `delete_optstamp` equal the latest opstamp of the tasks of the -/// corresponding index. -#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] -#[async_trait] -pub trait Metastore: fmt::Debug + Send + Sync + 'static { - /// Returns the metastore's uri. - fn uri(&self) -> &Uri; +static METASTORE_METRICS_LAYER: Lazy> = + Lazy::new(|| PrometheusMetricsLayer::new("metastore", ["request"])); - /// Checks whether the metastore is available. - async fn check_connectivity(&self) -> anyhow::Result<()>; - - // Index API - - /// Creates an index. - /// - /// This API creates a new index in the metastore. - /// An error will occur if an index that already exists in the storage is specified. - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult; +pub(crate) fn instrument_metastore( + metastore_impl: impl MetastoreService, +) -> MetastoreServiceClient { + MetastoreServiceClient::tower() + .shared_layer(METASTORE_METRICS_LAYER.clone()) + .build(metastore_impl) +} +/// An extended trait for [`MetastoreService`]. +#[async_trait] +pub trait MetastoreServiceExt: MetastoreService { /// Returns whether the index `index_id` exists in the metastore. - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - match self.index_metadata(index_id).await { + async fn index_exists(&mut self, index_id: &str) -> MetastoreResult { + let request = IndexMetadataRequest::for_index_id(index_id.to_string()); + match self.index_metadata(request).await { Ok(_) => Ok(true), Err(MetastoreError::NotFound { .. }) => Ok(false), Err(error) => Err(error), } } - - /// Returns index uid for the given index. - async fn index_uid(&self, index_id: &str) -> MetastoreResult { - let index_uid = self.index_metadata(index_id).await?.index_uid; - Ok(index_uid) - } - - /// Returns the [`IndexMetadata`] of an index identified by its ID. - /// TODO consider merging with list_splits to remove one round-trip - async fn index_metadata(&self, index_id: &str) -> MetastoreResult; - - /// Returns the [`IndexMetadata`] of an index identified by its UID. - async fn index_metadata_strict(&self, index_uid: &IndexUid) -> MetastoreResult { - let index_metadata = self.index_metadata(index_uid.index_id()).await?; - - if index_metadata.index_uid != *index_uid { - let index_id = index_uid.index_id().to_string(); - return Err(MetastoreError::NotFound(EntityKind::Index { index_id })); - } - Ok(index_metadata) - } - - /// Lists the indexes. - /// - /// This API lists the indexes stored in the metastore and returns a collection of - /// [`IndexMetadata`]. - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult>; - - /// Deletes an index. - /// - /// This API removes the specified from the metastore, but does not remove the index from the - /// storage. An error will occur if an index that does not exist in the storage is - /// specified. - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()>; - - // Split API - - /// Stages multiple splits. - /// - /// If a split already exists and is not in the `Staged` state, a `SplitsNotStaged` error - /// will be returned. - /// Attempting to re-stage any split which is not currently `Staged` is incorrect and should not - /// be attempted. - /// - /// A split needs to be staged before uploading any of its files to the storage. - /// An error will occur if an index that does not exist in the storage is specified. - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()>; - - /// Publishes a set of staged splits while optionally marking another set of published splits - /// for deletion. - /// - /// This API merely updates the state of the staged splits from [`SplitState::Staged`] to - /// [`SplitState::Published`]. At this point, the split files are assumed to have already - /// been uploaded. - /// An error will occur if you specify an index or split that does not exist in the storage. - /// - /// This method can be used to advance the checkpoint, by supplying an empty array for - /// `staged_split_ids`. - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()>; - - /// Lists the splits. - /// - /// Returns a list of splits that intersects the given `time_range`, `split_state`, and `tag`. - /// Regardless of the time range filter, if a split has no timestamp it is always returned. - /// An error will occur if an index that does not exist in the storage is specified. - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult>; - - /// Lists all the splits without filtering. - /// - /// Returns a list of all splits currently known to the metastore regardless of their state. - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - let query = ListSplitsQuery::for_index(index_uid); - self.list_splits(query).await - } - - /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_uid`. - /// These splits are called "stale" as they have an `delete_opstamp` strictly inferior - /// to the given `delete_opstamp`. - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - let query = ListSplitsQuery::for_index(index_uid) - .with_delete_opstamp_lt(delete_opstamp) - .with_split_state(SplitState::Published) - .retain_mature(OffsetDateTime::now_utc()); - - let mut splits = self.list_splits(query).await?; - splits.sort_by(|split_left, split_right| { - split_left - .split_metadata - .delete_opstamp - .cmp(&split_right.split_metadata.delete_opstamp) - .then_with(|| { - split_left - .publish_timestamp - .cmp(&split_right.publish_timestamp) - }) - }); - splits.truncate(num_splits); - Ok(splits) - } - - /// Marks a list of splits for deletion. - /// - /// This API will change the state to [`SplitState::MarkedForDeletion`] so that it is not - /// referenced by the client anymore. It actually does not remove the split from storage. An - /// error will occur if you specify an index or split that does not exist in the storage. - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()>; - - /// Deletes a list of splits. - /// - /// This API only accepts splits that are in [`SplitState::Staged`] or - /// [`SplitState::MarkedForDeletion`] state. This removes the split metadata from the - /// metastore, but does not remove the split from storage. An error will occur if you - /// specify an index or split that does not exist in the storage. - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()>; - - // Source API - - /// Adds a new source. Fails with [`MetastoreError::NotFound`] if a source with the same ID is - /// already defined for the index. - /// - /// If a checkpoint is already registered for the source, it is kept. - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()>; - - /// Enables or Disables a source. - /// Fails with `SourceDoesNotExist` error if the specified source doesn't exist. - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()>; - - /// Resets the checkpoint of a source identified by `index_uid` and `source_id`. - async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()>; - - /// Deletes a source. Fails with [`MetastoreError::NotFound`] if the specified source does not - /// exist. - /// - /// The checkpoint associated to the source is deleted as well. - /// If the checkpoint is missing, this does not trigger an error. - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()>; - - // Delete tasks API - - /// Creates a new [`DeleteTask`] from a [`DeleteQuery`]. - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult; - - /// Retrieves the last delete opstamp for a given `index_uid`. - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult; - - /// Updates splits `split_metadata.delete_opstamp` to the value `delete_opstamp`. - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()>; - - /// Lists [`DeleteTask`] with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. - async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult>; - - // Shard API: - // - `open_shards` - // - `close_shards` - // - `list_shards` - // - `delete_shards` - - /// Creates new open shards for one or multiple indexes. - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult; - - /// - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult; - - /// Closes some shards, i.e. changes their state from `Open` to `Closing` or `Closed`. - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> MetastoreResult; - - /// Lists the shards of one or multiple indexes. - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult; - - /// Deletes some shards. - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> MetastoreResult; } +impl MetastoreServiceExt for MetastoreServiceClient {} + /// A query object for listing indexes stored in the metastore. #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] pub enum ListIndexesQuery { @@ -357,8 +84,313 @@ pub enum ListIndexesQuery { IndexIdPatterns(Vec), } -/// A query object for listing splits stored in the metastore. +//// Helper trait to build a [`ListIndexesMetadataRequest`] +/// and deserialize its payload. +pub trait ListIndexesMetadataRequestExt { + /// Creates a new [`ListIndexesMetadataRequest`] from a [`ListIndexesQuery`]. + fn try_from_list_indexes_query( + list_indexes_query: ListIndexesQuery, + ) -> MetastoreResult; + + /// Creates a new [`ListIndexesMetadataRequest`] that matches all indexes. + fn all() -> ListIndexesMetadataRequest; + + /// Deserializes the `query_json` field of a [`ListIndexesMetadataRequest`] into a + /// [`ListIndexesQuery`]. + fn deserialize_list_indexes_query(&self) -> MetastoreResult; +} + +impl ListIndexesMetadataRequestExt for ListIndexesMetadataRequest { + fn try_from_list_indexes_query( + list_indexes_query: ListIndexesQuery, + ) -> MetastoreResult { + let query_json = serde_utils::to_json_str(&list_indexes_query)?; + Ok(Self { query_json }) + } + + fn all() -> ListIndexesMetadataRequest { + Self::try_from_list_indexes_query(ListIndexesQuery::All).expect("should never fail") + } + + fn deserialize_list_indexes_query(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.query_json) + } +} + +/// Helper trait to build a [`CreateIndexRequest`] and deserialize its payload. +pub trait CreateIndexRequestExt { + /// Creates a new [`CreateIndexRequest`] from an [`IndexConfig`]. + fn try_from_index_config(index_config: IndexConfig) -> MetastoreResult; + + /// Deserializes the `index_config_json` field of a [`CreateIndexRequest`] into an + /// [`IndexConfig`]. + fn deserialize_index_config(&self) -> MetastoreResult; +} + +impl CreateIndexRequestExt for CreateIndexRequest { + fn try_from_index_config(index_config: IndexConfig) -> MetastoreResult { + let index_config_json = serde_utils::to_json_str(&index_config)?; + let request = Self { index_config_json }; + Ok(request) + } + + fn deserialize_index_config(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.index_config_json) + } +} + +/// Helper trait to build a [`IndexMetadataResponse`] and deserialize its payload. +pub trait IndexMetadataResponseExt { + /// Creates a new [`IndexMetadataResponse`] from an [`IndexMetadata`]. + fn try_from_index_metadata( + index_metadata: IndexMetadata, + ) -> MetastoreResult; + + /// Deserializes the `index_metadata_serialized_json` field of a [`IndexMetadataResponse`] into + /// an [`IndexMetadata`]. + fn deserialize_index_metadata(&self) -> MetastoreResult; +} + +impl IndexMetadataResponseExt for IndexMetadataResponse { + fn try_from_index_metadata(index_metadata: IndexMetadata) -> MetastoreResult { + let index_metadata_serialized_json = serde_utils::to_json_str(&index_metadata)?; + let request = Self { + index_metadata_serialized_json, + }; + Ok(request) + } + + fn deserialize_index_metadata(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.index_metadata_serialized_json) + } +} + +/// Helper trait to build a `ListIndexesResponse` and deserialize its payload. +pub trait ListIndexesMetadataResponseExt { + /// Creates a new `ListIndexesResponse` from a list of [`IndexMetadata`]. + fn try_from_indexes_metadata( + indexes_metadata: impl IntoIterator, + ) -> MetastoreResult; + + /// Deserializes the `indexes_metadata_serialized_json` field of a `ListIndexesResponse` into + /// a list of [`IndexMetadata`]. + fn deserialize_indexes_metadata(&self) -> MetastoreResult>; + + /// Creates an empty `ListIndexesResponse`. + fn empty() -> Self; +} + +impl ListIndexesMetadataResponseExt for ListIndexesMetadataResponse { + fn try_from_indexes_metadata( + indexes_metadata: impl IntoIterator, + ) -> MetastoreResult { + let indexes_metadata: Vec = indexes_metadata.into_iter().collect(); + let indexes_metadata_serialized_json = serde_utils::to_json_str(&indexes_metadata)?; + let request = Self { + indexes_metadata_serialized_json, + }; + Ok(request) + } + + fn empty() -> Self { + Self { + indexes_metadata_serialized_json: "[]".to_string(), + } + } + + fn deserialize_indexes_metadata(&self) -> MetastoreResult> { + serde_utils::from_json_str(&self.indexes_metadata_serialized_json) + } +} + +/// Helper trait to build a [`AddSourceRequest`] and deserialize its payload. +pub trait AddSourceRequestExt { + /// Creates a new [`AddSourceRequest`] from a [`SourceConfig`]. + fn try_from_source_config( + index_uid: impl Into, + source_config: SourceConfig, + ) -> MetastoreResult; + + /// Deserializes the `source_config_json` field of a [`AddSourceRequest`] into a + /// [`SourceConfig`]. + fn deserialize_source_config(&self) -> MetastoreResult; +} + +impl AddSourceRequestExt for AddSourceRequest { + fn try_from_source_config( + index_uid: impl Into, + source_config: SourceConfig, + ) -> MetastoreResult { + let source_config_json = serde_utils::to_json_str(&source_config)?; + let request = Self { + index_uid: index_uid.into().into(), + source_config_json, + }; + Ok(request) + } + + fn deserialize_source_config(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.source_config_json) + } +} + +/// Helper trait to build a [`DeleteTask`] and deserialize its payload. +pub trait StageSplitsRequestExt { + /// Creates a new [`StageSplitsRequest`] from a [`SplitMetadata`]. + fn try_from_split_metadata( + index_uid: impl Into, + split_metadata: SplitMetadata, + ) -> MetastoreResult; + + /// Creates a new [`StageSplitsRequest`] from a list of [`SplitMetadata`]. + fn try_from_splits_metadata( + index_uid: impl Into, + splits_metadata: impl IntoIterator, + ) -> MetastoreResult; + + /// Deserializes the `split_metadata_list_serialized_json` field of a [`StageSplitsRequest`] + /// into a list of [`SplitMetadata`]. + fn deserialize_splits_metadata(&self) -> MetastoreResult>; +} + +impl StageSplitsRequestExt for StageSplitsRequest { + fn try_from_split_metadata( + index_uid: impl Into, + split_metadata: SplitMetadata, + ) -> MetastoreResult { + let split_metadata_list_serialized_json = serde_utils::to_json_str(&[split_metadata])?; + let request = Self { + index_uid: index_uid.into().into(), + split_metadata_list_serialized_json, + }; + Ok(request) + } + + fn try_from_splits_metadata( + index_uid: impl Into, + splits_metadata: impl IntoIterator, + ) -> MetastoreResult { + let splits_metadata: Vec = splits_metadata.into_iter().collect(); + let split_metadata_list_serialized_json = serde_utils::to_json_str(&splits_metadata)?; + let request = Self { + index_uid: index_uid.into().into(), + split_metadata_list_serialized_json, + }; + Ok(request) + } + + fn deserialize_splits_metadata(&self) -> MetastoreResult> { + serde_utils::from_json_str(&self.split_metadata_list_serialized_json) + } +} + +/// Helper trait to build a [`ListSplitsRequest`] and deserialize its payload. +pub trait ListSplitsRequestExt { + /// Creates a new [`ListSplitsRequest`] from an [`IndexUid`]. + fn try_from_index_uid(index_uid: IndexUid) -> MetastoreResult; + + /// Creates a new [`ListSplitsRequest`] from a [`ListSplitsQuery`]. + fn try_from_list_splits_query( + list_splits_query: ListSplitsQuery, + ) -> MetastoreResult; + + /// Deserializes the `query_json` field of a [`ListSplitsRequest`] into a [`ListSplitsQuery`]. + fn deserialize_list_splits_query(&self) -> MetastoreResult; +} + +impl ListSplitsRequestExt for ListSplitsRequest { + fn try_from_index_uid(index_uid: IndexUid) -> MetastoreResult { + let list_splits_query = ListSplitsQuery::for_index(index_uid); + Self::try_from_list_splits_query(list_splits_query) + } + + fn try_from_list_splits_query( + list_splits_query: ListSplitsQuery, + ) -> MetastoreResult { + let query_json = serde_utils::to_json_str(&list_splits_query)?; + let request = Self { query_json }; + Ok(request) + } + + fn deserialize_list_splits_query(&self) -> MetastoreResult { + let list_splits_query = serde_utils::from_json_str(&self.query_json)?; + Ok(list_splits_query) + } +} + +/// Helper trait to build a [`ListSplitsResponse`] and deserialize its payload. +pub trait ListSplitsResponseExt { + /// Creates a new [`ListSplitsResponse`] from a list of [`Split`]. + fn try_from_splits( + splits: impl IntoIterator, + ) -> MetastoreResult; + + /// Deserializes the `splits_serialized_json` field of a [`ListSplitsResponse`] into a list of + /// [`Split`]. + fn deserialize_splits(&self) -> MetastoreResult>; + + /// Deserializes the `splits_serialized_json` field of a [`ListSplitsResponse`] into a list of + /// [`SplitMetadata`]. + fn deserialize_splits_metadata(&self) -> MetastoreResult> { + let splits = self.deserialize_splits()?; + Ok(splits + .into_iter() + .map(|split| split.split_metadata) + .collect()) + } + + /// Deserializes the `splits_serialized_json` field of a [`ListSplitsResponse`] into a list of + /// [`SplitId`]. + fn deserialize_split_ids(&self) -> MetastoreResult> { + let splits = self.deserialize_splits()?; + Ok(splits + .into_iter() + .map(|split| split.split_metadata.split_id) + .collect()) + } + + /// Creates an empty [`ListSplitsResponse`]. + fn empty() -> Self; +} + +/// Helper trait for [`PublishSplitsRequest`] to deserialize its payload. +pub trait PublishSplitsRequestExt { + /// Deserializes the `index_checkpoint_delta_json_opt` field of a [`PublishSplitsRequest`] into + /// an [`Option`]. + fn deserialize_index_checkpoint(&self) -> MetastoreResult>; +} + +impl PublishSplitsRequestExt for PublishSplitsRequest { + fn deserialize_index_checkpoint(&self) -> MetastoreResult> { + self.index_checkpoint_delta_json_opt + .as_ref() + .map(|value| serde_utils::from_json_str(value)) + .transpose() + } +} + +impl ListSplitsResponseExt for ListSplitsResponse { + fn empty() -> Self { + Self { + splits_serialized_json: "[]".to_string(), + } + } + + fn try_from_splits(splits: impl IntoIterator) -> MetastoreResult { + let splits_serialized_json = serde_utils::to_json_str(&splits.into_iter().collect_vec())?; + let request = Self { + splits_serialized_json, + }; + Ok(request) + } + + fn deserialize_splits(&self) -> MetastoreResult> { + serde_utils::from_json_str(&self.splits_serialized_json) + } +} + #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] +/// A query builder for listing splits within the metastore. pub struct ListSplitsQuery { /// A non-empty list of index UIDs to get splits from. pub index_uids: Vec, @@ -389,6 +421,10 @@ pub struct ListSplitsQuery { /// The datetime at which you include or exclude mature splits. pub mature: Bound, + + /// Sorts the splits by staleness, i.e. by delete opstamp and publish timestamp in ascending + /// order. + pub sort_by_staleness: bool, } #[allow(unused_attributes)] @@ -406,6 +442,7 @@ impl ListSplitsQuery { update_timestamp: Default::default(), create_timestamp: Default::default(), mature: Bound::Unbounded, + sort_by_staleness: false, } } @@ -429,6 +466,7 @@ impl ListSplitsQuery { update_timestamp: Default::default(), create_timestamp: Default::default(), mature: Bound::Unbounded, + sort_by_staleness: false, }) } @@ -585,6 +623,13 @@ impl ListSplitsQuery { self.mature = Bound::Excluded(now); self } + + /// Sorts the splits by staleness, i.e. by delete opstamp and publish timestamp in ascending + /// order. + pub fn sort_by_staleness(mut self) -> Self { + self.sort_by_staleness = true; + self + } } #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] @@ -658,7 +703,7 @@ impl Default for FilterRange { } #[cfg(test)] -mod list_splits_query_tests { +mod tests { use super::*; #[test] @@ -747,4 +792,16 @@ mod list_splits_query_tests { assert!(!filter.overlaps_with(0..=49)); assert!(!filter.overlaps_with(75..=124)); } + + #[test] + fn test_list_splits_response_empty() { + let response = ListSplitsResponse::empty(); + assert_eq!(response.deserialize_splits().unwrap(), vec![]); + } + + #[test] + fn test_list_indexes_metadata_empty() { + let response = ListIndexesMetadataResponse::empty(); + assert_eq!(response.deserialize_indexes_metadata().unwrap(), vec![]); + } } diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index 11dbbcf1c34..b273de37c2e 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -29,17 +29,23 @@ use itertools::Itertools; use quickwit_common::uri::Uri; use quickwit_common::PrettySample; use quickwit_config::{ - validate_index_id_pattern, IndexConfig, MetastoreBackend, MetastoreConfig, - PostgresMetastoreConfig, SourceConfig, + validate_index_id_pattern, MetastoreBackend, MetastoreConfig, PostgresMetastoreConfig, }; use quickwit_doc_mapper::tag_pruning::TagFilterAst; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, - ListShardsRequest, ListShardsResponse, MetastoreError, MetastoreResult, OpenShardsRequest, - OpenShardsResponse, + AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CloseShardsRequest, + CloseShardsResponse, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, + DeleteShardsRequest, DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, + DeleteTask, EmptyResponse, EntityKind, IndexMetadataRequest, IndexMetadataResponse, + LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, + ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, + ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, + MetastoreService, MetastoreServiceClient, OpenShardsRequest, OpenShardsResponse, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::IndexUid; use sqlx::migrate::Migrator; use sqlx::postgres::{PgConnectOptions, PgDatabaseError, PgPoolOptions}; use sqlx::{ConnectOptions, Pool, Postgres, Transaction}; @@ -48,12 +54,14 @@ use tracing::log::LevelFilter; use tracing::{debug, error, info, instrument, warn}; use crate::checkpoint::IndexCheckpointDelta; -use crate::metastore::instrumented_metastore::InstrumentedMetastore; use crate::metastore::postgresql_model::{PgDeleteTask, PgIndex, PgSplit}; -use crate::metastore::FilterRange; +use crate::metastore::{instrument_metastore, FilterRange, PublishSplitsRequestExt}; use crate::{ - IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, MetastoreFactory, - MetastoreResolverError, Split, SplitMaturity, SplitMetadata, SplitState, + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, + ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, ListIndexesQuery, + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, MetastoreFactory, + MetastoreResolverError, MetastoreServiceExt, Split, SplitMaturity, SplitMetadata, SplitState, + StageSplitsRequestExt, }; static MIGRATOR: Migrator = sqlx::migrate!("migrations/postgresql"); @@ -466,17 +474,22 @@ where } #[async_trait] -impl Metastore for PostgresqlMetastore { - async fn check_connectivity(&self) -> anyhow::Result<()> { +impl MetastoreService for PostgresqlMetastore { + async fn check_connectivity(&mut self) -> anyhow::Result<()> { self.connection_pool.acquire().await?; Ok(()) } + fn endpoints(&self) -> Vec { + vec![self.uri.clone()] + } + #[instrument(skip(self))] - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> MetastoreResult { + let query = request.deserialize_list_indexes_query()?; let sql = match query { ListIndexesQuery::All => "SELECT * FROM indexes".to_string(), ListIndexesQuery::IndexIdPatterns(index_id_patterns) => { @@ -491,14 +504,20 @@ impl Metastore for PostgresqlMetastore { let pg_indexes = sqlx::query_as::<_, PgIndex>(&sql) .fetch_all(&self.connection_pool) .await?; - pg_indexes + let indexes_metadata = pg_indexes .into_iter() .map(|pg_index| pg_index.index_metadata()) - .collect() + .collect::>>()?; + let response = ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata)?; + Ok(response) } - #[instrument(skip(self), fields(index_id=&index_config.index_id))] - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { + #[instrument(skip(self))] + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> MetastoreResult { + let index_config = request.deserialize_index_config()?; let index_metadata = IndexMetadata::new(index_config); let index_metadata_json = serde_json::to_string(&index_metadata).map_err(|error| { MetastoreError::JsonSerializeError { @@ -515,11 +534,17 @@ impl Metastore for PostgresqlMetastore { .execute(&self.connection_pool) .await .map_err(|error| convert_sqlx_err(index_metadata.index_id(), error))?; - Ok(index_metadata.index_uid) + Ok(CreateIndexResponse { + index_uid: index_metadata.index_uid.to_string(), + }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { + #[instrument(skip_all, fields(index_id=request.index_uid))] + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); let delete_res = sqlx::query("DELETE FROM indexes WHERE index_uid = $1") .bind(index_uid.to_string()) .execute(&self.connection_pool) @@ -529,15 +554,16 @@ impl Metastore for PostgresqlMetastore { index_id: index_uid.index_id().to_string(), })); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self, split_metadata_list), fields(split_ids))] + #[instrument(skip_all, fields(split_ids))] async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { + &mut self, + request: StageSplitsRequest, + ) -> MetastoreResult { + let split_metadata_list = request.deserialize_splits_metadata()?; + let index_uid: IndexUid = request.index_uid.into(); let mut split_ids = Vec::with_capacity(split_metadata_list.len()); let mut time_range_start_list = Vec::with_capacity(split_metadata_list.len()); let mut time_range_end_list = Vec::with_capacity(split_metadata_list.len()); @@ -630,19 +656,20 @@ impl Metastore for PostgresqlMetastore { debug!(index_id=%index_uid.index_id(), num_splits=split_ids.len(), "Splits successfully staged."); - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - _publish_token_opt: Option, - ) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> MetastoreResult { + let checkpoint_delta_opt: Option = + request.deserialize_index_checkpoint()?; + let index_uid: IndexUid = request.index_uid.into(); + let staged_split_ids = request.staged_split_ids; + let replaced_split_ids = request.replaced_split_ids; run_with_tx!(self.connection_pool, tx, { let mut index_metadata = index_metadata(tx, index_uid.index_id()).await?; if index_metadata.index_uid != index_uid { @@ -777,12 +804,16 @@ impl Metastore for PostgresqlMetastore { "Published {} splits and marked {} splits for deletion successfully.", num_published_splits, num_marked_splits ); - Ok(()) + Ok(EmptyResponse {}) }) } - #[instrument(skip(self), fields(index_uids=query.index_uids.iter().join(",")))] - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { + #[instrument(skip(self))] + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> MetastoreResult { + let query = request.deserialize_list_splits_query()?; let sql_base = "SELECT * FROM splits".to_string(); let sql = build_query_filter(sql_base, &query); @@ -800,9 +831,14 @@ impl Metastore for PostgresqlMetastore { .iter() .map(|index_uid| index_uid.index_id().to_string()) .collect(); + let list_indexes_metadata_request = + ListIndexesMetadataRequest::try_from_list_indexes_query( + ListIndexesQuery::IndexIdPatterns(index_ids_str.clone()), + )?; let found_index_ids: HashSet = self - .list_indexes_metadatas(ListIndexesQuery::IndexIdPatterns(index_ids_str.clone())) + .list_indexes_metadata(list_indexes_metadata_request) .await? + .deserialize_indexes_metadata()? .into_iter() .map(|index_metadata| index_metadata.index_id().to_string()) .collect(); @@ -816,18 +852,21 @@ impl Metastore for PostgresqlMetastore { })); } } - pg_splits + let splits = pg_splits .into_iter() .map(|pg_split| pg_split.try_into()) - .collect() + .collect::>>()?; + let response = ListSplitsResponse::try_from_splits(splits)?; + Ok(response) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + let split_ids = request.split_ids; const MARK_SPLITS_FOR_DELETION_QUERY: &str = r#" -- Select the splits to update, regardless of their state. -- The left join make it possible to identify the splits that do not exist. @@ -866,7 +905,7 @@ impl Metastore for PostgresqlMetastore { let (num_found_splits, num_marked_splits, not_found_split_ids): (i64, i64, Vec) = sqlx::query_as(MARK_SPLITS_FOR_DELETION_QUERY) .bind(index_uid.to_string()) - .bind(split_ids) + .bind(split_ids.clone()) .fetch_one(&self.connection_pool) .await .map_err(|error| convert_sqlx_err(index_uid.index_id(), error))?; @@ -894,15 +933,16 @@ impl Metastore for PostgresqlMetastore { not_found_split_ids.len() ); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + let split_ids = request.split_ids; const DELETE_SPLITS_QUERY: &str = r#" -- Select the splits to delete, regardless of their state. -- The left join make it possible to identify the splits that do not exist. @@ -982,86 +1022,109 @@ impl Metastore for PostgresqlMetastore { not_found_split_ids.len() ); } - Ok(()) + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_id))] - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - index_opt(&self.connection_pool, index_id) - .await? - .ok_or_else(|| { + #[instrument(skip(self))] + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> MetastoreResult { + let response = if let Some(index_id) = &request.index_id { + index_opt(&self.connection_pool, index_id).await? + } else if let Some(index_uid) = &request.index_uid { + let index_uid: IndexUid = index_uid.to_string().into(); + index_opt_for_uid(&self.connection_pool, index_uid).await? + } else { + return Err(MetastoreError::Internal { + message: "either `index_id` or `index_uid` must be set".to_string(), + cause: "missing index identifier".to_string(), + }); + }; + let index_metadata = response + .ok_or({ MetastoreError::NotFound(EntityKind::Index { - index_id: index_id.to_string(), + index_id: request.get_index_id().expect("index_id is set").to_string(), }) })? - .index_metadata() + .index_metadata()?; + let response = IndexMetadataResponse::try_from_index_metadata(index_metadata)?; + Ok(response) } - #[instrument(skip(self, source), fields(index_id=index_uid.index_id(), source_id=source.source_id))] - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid.into(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata::( tx, index_uid, |index_metadata: &mut IndexMetadata| { - index_metadata.add_source(source)?; + index_metadata.add_source(source_config)?; Ok(true) }, ) .await?; Ok(()) - }) + })?; + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] + #[instrument(skip(self))] async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { + &mut self, + request: ToggleSourceRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - index_metadata.toggle_source(source_id, enable) + index_metadata.toggle_source(&request.source_id, request.enable) }) .await?; Ok(()) - }) + })?; + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - index_metadata.delete_source(source_id) + index_metadata.delete_source(&request.source_id) }) .await?; Ok(()) - }) + })?; + Ok(EmptyResponse {}) } - #[instrument(skip(self), fields(index_id=index_uid.index_id(), source_id=source_id))] + #[instrument(skip(self))] async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { + &mut self, + request: ResetSourceCheckpointRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { - Ok::<_, MetastoreError>(index_metadata.checkpoint.reset_source(source_id)) + Ok::<_, MetastoreError>(index_metadata.checkpoint.reset_source(&request.source_id)) }) .await?; Ok(()) - }) - } - - fn uri(&self) -> &Uri { - &self.uri + })?; + Ok(EmptyResponse {}) } /// Retrieves the last delete opstamp for a given `index_id`. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { + #[instrument(skip(self))] + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> MetastoreResult { let max_opstamp: i64 = sqlx::query_scalar( r#" SELECT COALESCE(MAX(opstamp), 0) @@ -1069,19 +1132,22 @@ impl Metastore for PostgresqlMetastore { WHERE index_uid = $1 "#, ) - .bind(index_uid.to_string()) + .bind(request.index_uid) .fetch_one(&self.connection_pool) .await .map_err(|error| MetastoreError::Db { message: error.to_string(), })?; - Ok(max_opstamp as u64) + Ok(LastDeleteOpstampResponse::new(max_opstamp as u64)) } /// Creates a delete task from a delete query. - #[instrument(skip(self), fields(index_id=IndexUid::from(delete_query.index_uid.to_string()).index_id()))] - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { + #[instrument(skip(self))] + async fn create_delete_task( + &mut self, + delete_query: DeleteQuery, + ) -> MetastoreResult { let delete_query_json = serde_json::to_string(&delete_query).map_err(|error| { MetastoreError::JsonSerializeError { struct_name: "DeleteQuery".to_string(), @@ -1114,15 +1180,15 @@ impl Metastore for PostgresqlMetastore { } /// Update splits delete opstamps. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { + #[instrument(skip(self))] + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); + let split_ids = request.split_ids; if split_ids.is_empty() { - return Ok(()); + return Ok(UpdateSplitsDeleteOpstampResponse {}); } let update_res = sqlx::query( r#" @@ -1139,7 +1205,7 @@ impl Metastore for PostgresqlMetastore { AND split_id = ANY($3) "#, ) - .bind(delete_opstamp as i64) + .bind(request.delete_opstamp as i64) .bind(index_uid.to_string()) .bind(split_ids) .execute(&self.connection_pool) @@ -1155,16 +1221,16 @@ impl Metastore for PostgresqlMetastore { index_id: index_uid.index_id().to_string(), })); } - Ok(()) + Ok(UpdateSplitsDeleteOpstampResponse {}) } /// Lists the delete tasks with opstamp > `opstamp_start`. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] + #[instrument(skip(self))] async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { + &mut self, + request: ListDeleteTasksRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); let pg_delete_tasks: Vec = sqlx::query_as::<_, PgDeleteTask>( r#" SELECT * FROM delete_tasks @@ -1174,25 +1240,25 @@ impl Metastore for PostgresqlMetastore { "#, ) .bind(index_uid.to_string()) - .bind(opstamp_start as i64) + .bind(request.opstamp_start as i64) .fetch_all(&self.connection_pool) .await?; - pg_delete_tasks + let delete_tasks = pg_delete_tasks .into_iter() .map(|pg_delete_task| pg_delete_task.try_into()) - .collect() + .collect::>>()?; + Ok(ListDeleteTasksResponse { delete_tasks }) } /// Returns `num_splits` published splits with `split.delete_opstamp` < `delete_opstamp`. /// Results are ordered by ascending `split.delete_opstamp` and `split.publish_timestamp` /// values. - #[instrument(skip(self), fields(index_id=index_uid.index_id()))] + #[instrument(skip(self))] async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { + &mut self, + request: ListStaleSplitsRequest, + ) -> MetastoreResult { + let index_uid: IndexUid = request.index_uid.into(); let pg_stale_splits: Vec = sqlx::query_as::<_, PgSplit>( r#" SELECT * @@ -1207,9 +1273,9 @@ impl Metastore for PostgresqlMetastore { "#, ) .bind(index_uid.to_string()) - .bind(delete_opstamp as i64) + .bind(request.delete_opstamp as i64) .bind(SplitState::Published.as_str()) - .bind(num_splits as i64) + .bind(request.num_splits as i64) .fetch_all(&self.connection_pool) .await?; @@ -1223,48 +1289,52 @@ impl Metastore for PostgresqlMetastore { index_id: index_uid.index_id().to_string(), })); } - pg_stale_splits + let splits = pg_stale_splits .into_iter() .map(|pg_split| pg_split.try_into()) - .collect() + .collect::>>()?; + let response = ListSplitsResponse::try_from_splits(splits)?; + Ok(response) } async fn open_shards( - &self, + &mut self, _request: OpenShardsRequest, ) -> MetastoreResult { unimplemented!("`open_shards` is not implemented for PostgreSQL metastore") } async fn acquire_shards( - &self, + &mut self, _request: AcquireShardsRequest, ) -> MetastoreResult { unimplemented!("`close_shards` is not implemented for PostgreSQL metastore") } async fn close_shards( - &self, + &mut self, _request: CloseShardsRequest, ) -> MetastoreResult { unimplemented!("`close_shards` is not implemented for PostgreSQL metastore") } async fn list_shards( - &self, + &mut self, _request: ListShardsRequest, ) -> MetastoreResult { unimplemented!("`list_shards` is not implemented for PostgreSQL metastore") } async fn delete_shards( - &self, + &mut self, _request: DeleteShardsRequest, ) -> MetastoreResult { unimplemented!("`delete_shards` is not implemented for PostgreSQL metastore") } } +impl MetastoreServiceExt for PostgresqlMetastore {} + // We use dollar-quoted strings in Postgresql. // // In order to ensure that we do not risk SQL injection, @@ -1356,13 +1426,13 @@ pub struct PostgresqlMetastoreFactory { // In contrast to the file backe metastore, we use a strong pointer here, so that Metastore // doesn't get dropped. This is done in order to keep the underlying connection pool to // postgres alive. - cache: Arc>>>, + cache: Arc>>, } impl PostgresqlMetastoreFactory { - async fn get_from_cache(&self, uri: &Uri) -> Option> { + async fn get_from_cache(&self, uri: &Uri) -> Option { let cache_lock = self.cache.lock().await; - cache_lock.get(uri).map(Arc::clone) + cache_lock.get(uri).map(MetastoreServiceClient::clone) } /// If there is a valid entry in the cache to begin with, we trash the new @@ -1370,7 +1440,11 @@ impl PostgresqlMetastoreFactory { /// /// This way we make sure that we keep only one instance associated /// to the key `uri` outside of this struct. - async fn cache_metastore(&self, uri: Uri, metastore: Arc) -> Arc { + async fn cache_metastore( + &self, + uri: Uri, + metastore: MetastoreServiceClient, + ) -> MetastoreServiceClient { let mut cache_lock = self.cache.lock().await; if let Some(metastore) = cache_lock.get(&uri) { return metastore.clone(); @@ -1390,7 +1464,7 @@ impl MetastoreFactory for PostgresqlMetastoreFactory { &self, metastore_config: &MetastoreConfig, uri: &Uri, - ) -> Result, MetastoreResolverError> { + ) -> Result { if let Some(metastore) = self.get_from_cache(uri).await { debug!("using metastore from cache"); return Ok(metastore); @@ -1406,9 +1480,9 @@ impl MetastoreFactory for PostgresqlMetastoreFactory { let postgresql_metastore = PostgresqlMetastore::new(postgresql_metastore_config, uri) .await .map_err(MetastoreResolverError::Initialization)?; - let instrumented_metastore = InstrumentedMetastore::new(Box::new(postgresql_metastore)); + let instrumented_metastore = instrument_metastore(postgresql_metastore); let unique_metastore_for_uri = self - .cache_metastore(uri.clone(), Arc::new(instrumented_metastore)) + .cache_metastore(uri.clone(), instrumented_metastore) .await; Ok(unique_metastore_for_uri) } @@ -1446,13 +1520,22 @@ metastore_test_suite!(crate::PostgresqlMetastore); #[cfg(test)] mod tests { use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; + use quickwit_proto::metastore::MetastoreService; use quickwit_proto::IndexUid; use time::OffsetDateTime; - use super::{build_query_filter, tags_filter_expression_helper}; + use super::{build_query_filter, tags_filter_expression_helper, PostgresqlMetastore}; use crate::metastore::postgresql_metastore::build_index_id_patterns_sql_query; + use crate::tests::test_suite::DefaultForTest; use crate::{ListSplitsQuery, SplitState}; + #[tokio::test] + async fn test_metastore_connectivity_and_endpoints() { + let mut metastore = PostgresqlMetastore::default_for_test().await; + metastore.check_connectivity().await.unwrap(); + assert!(metastore.endpoints()[0].protocol().is_postgresql()); + } + fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: &str) { assert_eq!(tags_filter_expression_helper(&tags_ast), expected); } diff --git a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs deleted file mode 100644 index 442dcd43c78..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/mod.rs +++ /dev/null @@ -1,331 +0,0 @@ -// 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 . - -mod retry; -#[cfg(test)] -mod test; - -use std::fmt; - -use async_trait::async_trait; -use quickwit_common::retry::RetryParams; -use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, ListShardsRequest, - ListShardsResponse, MetastoreResult, OpenShardsRequest, OpenShardsResponse, -}; -use quickwit_proto::{IndexUid, PublishToken}; - -use self::retry::retry; -use crate::checkpoint::IndexCheckpointDelta; -use crate::{IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMetadata}; - -/// Retry layer for a [`Metastore`]. -/// This is a band-aid solution for now. This will be removed after retry can be usable on -/// tonic level. -/// Tracking Issue: -pub struct RetryingMetastore { - inner: Box, - retry_params: RetryParams, -} - -impl RetryingMetastore { - /// Creates a retry layer for a [`Metastore`] - pub fn new(metastore: Box) -> Self { - Self { - inner: metastore, - retry_params: RetryParams { - max_attempts: 5, - ..Default::default() - }, - } - } -} - -impl fmt::Debug for RetryingMetastore { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("RetryingMetastore").finish() - } -} - -#[async_trait] -impl Metastore for RetryingMetastore { - fn uri(&self) -> &Uri { - self.inner.uri() - } - - async fn check_connectivity(&self) -> anyhow::Result<()> { - self.inner.check_connectivity().await - } - - async fn create_index(&self, index_config: IndexConfig) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.create_index(index_config.clone()).await - }) - .await - } - - async fn index_exists(&self, index_id: &str) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.index_exists(index_id).await - }) - .await - } - - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.index_metadata(index_id).await - }) - .await - } - - async fn list_indexes_metadatas( - &self, - query: ListIndexesQuery, - ) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner.list_indexes_metadatas(query.clone()).await - }) - .await - } - - async fn delete_index(&self, index_uid: IndexUid) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner.delete_index(index_uid.clone()).await - }) - .await - } - - async fn stage_splits( - &self, - index_uid: IndexUid, - split_metadata_list: Vec, - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .stage_splits(index_uid.clone(), split_metadata_list.clone()) - .await - }) - .await - } - - async fn publish_splits<'a>( - &self, - index_uid: IndexUid, - staged_split_ids: &[&'a str], - replaced_split_ids: &[&'a str], - checkpoint_delta_opt: Option, - publish_token_opt: Option, - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .publish_splits( - index_uid.clone(), - staged_split_ids, - replaced_split_ids, - checkpoint_delta_opt.clone(), - publish_token_opt.clone(), - ) - .await - }) - .await - } - - async fn list_splits(&self, query: ListSplitsQuery) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner.list_splits(query.clone()).await - }) - .await - } - - async fn list_all_splits(&self, index_uid: IndexUid) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner.list_all_splits(index_uid.clone()).await - }) - .await - } - - async fn list_stale_splits( - &self, - index_uid: IndexUid, - delete_opstamp: u64, - num_splits: usize, - ) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner - .list_stale_splits(index_uid.clone(), delete_opstamp, num_splits) - .await - }) - .await - } - - async fn mark_splits_for_deletion<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .mark_splits_for_deletion(index_uid.clone(), split_ids) - .await - }) - .await - } - - async fn delete_splits<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner.delete_splits(index_uid.clone(), split_ids).await - }) - .await - } - - async fn add_source(&self, index_uid: IndexUid, source: SourceConfig) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .add_source(index_uid.clone(), source.clone()) - .await - }) - .await - } - - async fn toggle_source( - &self, - index_uid: IndexUid, - source_id: &str, - enable: bool, - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .toggle_source(index_uid.clone(), source_id, enable) - .await - }) - .await - } - - async fn reset_source_checkpoint( - &self, - index_uid: IndexUid, - source_id: &str, - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .reset_source_checkpoint(index_uid.clone(), source_id) - .await - }) - .await - } - - async fn delete_source(&self, index_uid: IndexUid, source_id: &str) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner.delete_source(index_uid.clone(), source_id).await - }) - .await - } - - async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.create_delete_task(delete_query.clone()).await - }) - .await - } - - async fn last_delete_opstamp(&self, index_uid: IndexUid) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.last_delete_opstamp(index_uid.clone()).await - }) - .await - } - - async fn update_splits_delete_opstamp<'a>( - &self, - index_uid: IndexUid, - split_ids: &[&'a str], - delete_opstamp: u64, - ) -> MetastoreResult<()> { - retry(&self.retry_params, || async { - self.inner - .update_splits_delete_opstamp(index_uid.clone(), split_ids, delete_opstamp) - .await - }) - .await - } - - async fn list_delete_tasks( - &self, - index_uid: IndexUid, - opstamp_start: u64, - ) -> MetastoreResult> { - retry(&self.retry_params, || async { - self.inner - .list_delete_tasks(index_uid.clone(), opstamp_start) - .await - }) - .await - } - - async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.open_shards(request.clone()).await - }) - .await - } - - async fn acquire_shards( - &self, - request: AcquireShardsRequest, - ) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.acquire_shards(request.clone()).await - }) - .await - } - - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.close_shards(request.clone()).await - }) - .await - } - - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.list_shards(request.clone()).await - }) - .await - } - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> MetastoreResult { - retry(&self.retry_params, || async { - self.inner.delete_shards(request.clone()).await - }) - .await - } -} diff --git a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/retry.rs b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/retry.rs deleted file mode 100644 index 4e27fb5dd3b..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/retry.rs +++ /dev/null @@ -1,141 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use std::fmt::Debug; - -use futures::Future; -use quickwit_common::retry::{RetryParams, Retryable}; -use tracing::{debug, warn}; - -/// Retry with exponential backoff and full jitter. Implementation and default values originate from -/// the Java SDK. See also: . -pub async fn retry(retry_params: &RetryParams, f: F) -> Result -where - F: Fn() -> Fut, - Fut: Future>, - E: Retryable + Debug + 'static, -{ - let mut num_attempts = 0; - - loop { - let response = f().await; - - num_attempts += 1; - - match response { - Ok(response) => { - return Ok(response); - } - Err(error) => { - if !error.is_retryable() { - return Err(error); - } - if num_attempts >= retry_params.max_attempts { - warn!( - num_attempts=%num_attempts, - "Request failed" - ); - return Err(error); - } - let delay = retry_params.compute_delay(num_attempts); - debug!( - num_attempts=%num_attempts, - delay_millis=%delay.as_millis(), - error=?error, - "Request failed, retrying" - ); - tokio::time::sleep(delay).await; - } - } - } -} - -#[cfg(test)] -mod tests { - use std::sync::RwLock; - - use futures::future::ready; - - use super::{retry, RetryParams, Retryable}; - - #[derive(Debug, Eq, PartialEq)] - pub enum Retry { - Transient(E), - Permanent(E), - } - - impl Retryable for Retry { - fn is_retryable(&self) -> bool { - match self { - Retry::Transient(_) => true, - Retry::Permanent(_) => false, - } - } - } - - async fn simulate_retries(values: Vec>>) -> Result> { - let values_it = RwLock::new(values.into_iter()); - retry(&RetryParams::for_test(), || { - ready(values_it.write().unwrap().next().unwrap()) - }) - .await - } - - #[tokio::test] - async fn test_retry_accepts_ok() { - assert_eq!(simulate_retries(vec![Ok(())]).await, Ok(())); - } - - #[tokio::test] - async fn test_retry_does_retry() { - assert_eq!( - simulate_retries(vec![Err(Retry::Transient(1)), Ok(())]).await, - Ok(()) - ); - } - - #[tokio::test] - async fn test_retry_stops_retrying_on_non_retryable_error() { - assert_eq!( - simulate_retries(vec![Err(Retry::Permanent(1)), Ok(())]).await, - Err(Retry::Permanent(1)) - ); - } - - #[tokio::test] - async fn test_retry_retries_up_at_most_attempts_times() { - let retry_sequence: Vec<_> = (0..30) - .map(|retry_id| Err(Retry::Transient(retry_id))) - .chain(Some(Ok(()))) - .collect(); - assert_eq!( - simulate_retries(retry_sequence).await, - Err(Retry::Transient(29)) - ); - } - - #[tokio::test] - async fn test_retry_retries_up_to_max_attempts_times() { - let retry_sequence: Vec<_> = (0..29) - .map(|retry_id| Err(Retry::Transient(retry_id))) - .chain(Some(Ok(()))) - .collect(); - assert_eq!(simulate_retries(retry_sequence).await, Ok(())); - } -} diff --git a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs b/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs deleted file mode 100644 index f19e08e72e4..00000000000 --- a/quickwit/quickwit-metastore/src/metastore/retrying_metastore/test.rs +++ /dev/null @@ -1,374 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use std::fmt; -use std::sync::atomic::{AtomicUsize, Ordering}; - -use async_trait::async_trait; -use quickwit_common::retry::RetryParams; -use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, CloseShardsRequest, CloseShardsResponse, - DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, - ListShardsRequest, ListShardsResponse, MetastoreError, MetastoreResult, OpenShardsRequest, - OpenShardsResponse, -}; -use quickwit_proto::{IndexUid, PublishToken}; - -use crate::checkpoint::IndexCheckpointDelta; -use crate::{ - IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, RetryingMetastore, Split, - SplitMetadata, -}; - -struct RetryTestMetastore { - retry_count: AtomicUsize, - error_count: usize, - errors_to_return: Vec, -} - -impl fmt::Debug for RetryTestMetastore { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("RetryTestMetastore").finish() - } -} - -impl RetryTestMetastore { - fn new_with_errors(errors: &[MetastoreError]) -> Self { - Self { - retry_count: AtomicUsize::new(0), - error_count: errors.len(), - errors_to_return: errors.to_vec(), - } - } - - fn new_retrying_with_errors( - max_attempts: usize, - errors: &[MetastoreError], - ) -> RetryingMetastore { - RetryingMetastore { - inner: Box::new(RetryTestMetastore::new_with_errors(errors)), - retry_params: RetryParams { - max_attempts, - ..Default::default() - }, - } - } - - fn try_success(&self) -> MetastoreResult<()> { - let retry_count = self.retry_count.load(Ordering::SeqCst); - if retry_count < self.error_count { - self.retry_count.fetch_add(1, Ordering::SeqCst); - Err(self.errors_to_return[retry_count].clone()) - } else { - Ok(()) - } - } -} - -#[async_trait] -impl Metastore for RetryTestMetastore { - fn uri(&self) -> &Uri { - unimplemented!() - } - - async fn check_connectivity(&self) -> anyhow::Result<()> { - self.try_success().map_err(anyhow::Error::from) - } - - async fn create_index(&self, _index_config: IndexConfig) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(IndexUid::new("")), - Err(err) => Err(err), - } - } - - async fn index_metadata(&self, index_id: &str) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(IndexMetadata::for_test(index_id, "")), - Err(err) => Err(err), - } - } - - async fn list_indexes_metadatas( - &self, - _query: ListIndexesQuery, - ) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } - } - - async fn delete_index(&self, _index_uid: IndexUid) -> MetastoreResult<()> { - self.try_success() - } - - async fn stage_splits( - &self, - _index_uid: IndexUid, - _split_metadata_list: Vec, - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn publish_splits<'a>( - &self, - _index_uid: IndexUid, - _staged_split_ids: &[&'a str], - _replaced_split_ids: &[&'a str], - _checkpoint_delta_opt: Option, - _publish_token_opt: Option, - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn list_splits(&self, _query: ListSplitsQuery) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } - } - - async fn mark_splits_for_deletion<'a>( - &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn delete_splits<'a>( - &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn add_source(&self, _index_uid: IndexUid, _source: SourceConfig) -> MetastoreResult<()> { - self.try_success() - } - - async fn toggle_source( - &self, - _index_uid: IndexUid, - _source_id: &str, - _enable: bool, - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn reset_source_checkpoint( - &self, - _index_uid: IndexUid, - _source_id: &str, - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn delete_source(&self, _index_uid: IndexUid, _source_id: &str) -> MetastoreResult<()> { - self.try_success() - } - - async fn create_delete_task(&self, _delete_query: DeleteQuery) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(DeleteTask { - create_timestamp: 0, - opstamp: 0, - delete_query: None, - }), - Err(err) => Err(err), - } - } - - async fn last_delete_opstamp(&self, _index_uid: IndexUid) -> MetastoreResult { - let result = self.try_success(); - match result { - Ok(_) => Ok(0), - Err(err) => Err(err), - } - } - - async fn update_splits_delete_opstamp<'a>( - &self, - _index_uid: IndexUid, - _split_ids: &[&'a str], - _delete_opstamp: u64, - ) -> MetastoreResult<()> { - self.try_success() - } - - async fn list_delete_tasks( - &self, - _index_uid: IndexUid, - _opstamp_start: u64, - ) -> MetastoreResult> { - let result = self.try_success(); - match result { - Ok(_) => Ok(Vec::new()), - Err(err) => Err(err), - } - } - - async fn open_shards( - &self, - _request: OpenShardsRequest, - ) -> MetastoreResult { - self.try_success().map(|_| Default::default()) - } - - async fn acquire_shards( - &self, - _request: AcquireShardsRequest, - ) -> MetastoreResult { - self.try_success().map(|_| Default::default()) - } - - async fn close_shards( - &self, - _request: CloseShardsRequest, - ) -> MetastoreResult { - self.try_success().map(|_| Default::default()) - } - - async fn list_shards( - &self, - _request: ListShardsRequest, - ) -> MetastoreResult { - self.try_success().map(|_| Default::default()) - } - - async fn delete_shards( - &self, - _request: DeleteShardsRequest, - ) -> MetastoreResult { - self.try_success().map(|_| Default::default()) - } -} - -#[tokio::test] -async fn test_retryable_metastore_errors() { - let metastore: RetryingMetastore = RetryTestMetastore::new_retrying_with_errors( - 5, - &[ - MetastoreError::Connection { - message: "".to_string(), - }, - MetastoreError::Io { - message: "".to_string(), - }, - MetastoreError::Db { - message: "".to_string(), - }, - MetastoreError::Internal { - message: "".to_string(), - cause: "".to_string(), - }, - ], - ); - - // On retryable errors, if max retry count is not achieved, RetryingMetastore should retry until - // success - assert!(metastore - .list_indexes_metadatas(ListIndexesQuery::All) - .await - .is_ok()); - - let metastore: RetryingMetastore = RetryTestMetastore::new_retrying_with_errors( - 5, - &[MetastoreError::NotFound(EntityKind::Index { - index_id: "".to_string(), - })], - ); - - // On non-retryable errors, RetryingMetastore should exit with an error. - assert!(metastore - .list_indexes_metadatas(ListIndexesQuery::All) - .await - .is_err()); -} - -#[tokio::test] -async fn test_retryable_more_than_max_retry() { - let metastore: RetryingMetastore = RetryTestMetastore::new_retrying_with_errors( - 3, - &(0..4) - .collect::>() - .iter() - .map(|index| MetastoreError::Connection { - message: format!("{index}"), - }) - .collect::>(), - ); - - let error = metastore - .list_indexes_metadatas(ListIndexesQuery::All) - .await - .unwrap_err(); - assert_eq!( - error, - MetastoreError::Connection { - message: "2".to_string() // Max 3 retries, last error index is 2 - } - ) -} - -#[tokio::test] -async fn test_mixed_retryable_metastore_errors() { - let metastore: RetryingMetastore = RetryTestMetastore::new_retrying_with_errors( - 5, - &[ - MetastoreError::Connection { - message: "".to_string(), - }, - MetastoreError::Io { - message: "".to_string(), - }, - // Non-retryable - MetastoreError::AlreadyExists(EntityKind::Source { - index_id: "".to_string(), - source_id: "".to_string(), - }), - MetastoreError::Internal { - message: "".to_string(), - cause: "".to_string(), - }, - ], - ); - - let error = metastore - .list_indexes_metadatas(ListIndexesQuery::All) - .await - .unwrap_err(); - - assert_eq!( - error, - MetastoreError::AlreadyExists(EntityKind::Source { - index_id: "".to_string(), - source_id: "".to_string() - }), - ) -} diff --git a/quickwit/quickwit-metastore/src/metastore_factory.rs b/quickwit/quickwit-metastore/src/metastore_factory.rs index 40ef420d6ad..15a7c545b01 100644 --- a/quickwit/quickwit-metastore/src/metastore_factory.rs +++ b/quickwit/quickwit-metastore/src/metastore_factory.rs @@ -17,28 +17,27 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; - use async_trait::async_trait; use quickwit_common::uri::Uri; use quickwit_config::{MetastoreBackend, MetastoreConfig}; +use quickwit_proto::metastore::MetastoreServiceClient; -use crate::{Metastore, MetastoreResolverError}; +use crate::MetastoreResolverError; -/// A metastore factory builds a [`Metastore`] object for a target [`MetastoreBackend`] from a -/// [`MetastoreConfig`] and a [`Uri`]. +/// A metastore factory builds a [`MetastoreServiceClient`] object for a target [`MetastoreBackend`] +/// from a [`MetastoreConfig`] and a [`Uri`]. #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait] pub trait MetastoreFactory: Send + Sync + 'static { /// Returns the metastore backend targeted by the factory. fn backend(&self) -> MetastoreBackend; - /// Returns the appropriate [`Metastore`] object for the `uri`. + /// Returns the appropriate [`MetastoreServiceClient`] object for the `uri`. async fn resolve( &self, metastore_config: &MetastoreConfig, uri: &Uri, - ) -> Result, MetastoreResolverError>; + ) -> Result; } /// A metastore factory for handling unsupported or unavailable metastore backends. @@ -65,7 +64,7 @@ impl MetastoreFactory for UnsupportedMetastore { &self, _metastore_config: &MetastoreConfig, _uri: &Uri, - ) -> Result, MetastoreResolverError> { + ) -> Result { Err(MetastoreResolverError::UnsupportedBackend( self.message.to_string(), )) diff --git a/quickwit/quickwit-metastore/src/metastore_resolver.rs b/quickwit/quickwit-metastore/src/metastore_resolver.rs index 87b3b03c926..b03cc6ba11b 100644 --- a/quickwit/quickwit-metastore/src/metastore_resolver.rs +++ b/quickwit/quickwit-metastore/src/metastore_resolver.rs @@ -25,18 +25,19 @@ use anyhow::ensure; use once_cell::sync::Lazy; use quickwit_common::uri::{Protocol, Uri}; use quickwit_config::{MetastoreBackend, MetastoreConfig, MetastoreConfigs}; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_storage::StorageResolver; use crate::metastore::file_backed_metastore::FileBackedMetastoreFactory; #[cfg(feature = "postgres")] use crate::metastore::postgresql_metastore::PostgresqlMetastoreFactory; -use crate::{Metastore, MetastoreFactory, MetastoreResolverError}; +use crate::{MetastoreFactory, MetastoreResolverError}; type FactoryAndConfig = (Box, MetastoreConfig); -/// Returns the [`Metastore`] instance associated with the protocol of a URI. The actual creation of -/// metastore objects is delegated to pre-registered [`MetastoreFactory`]. The resolver is only -/// responsible for dispatching to the appropriate factory. +/// Returns the [`MetastoreServiceClient`] instance associated with the protocol of a URI. The +/// actual creation of metastore objects is delegated to pre-registered [`MetastoreFactory`]. The +/// resolver is only responsible for dispatching to the appropriate factory. #[derive(Clone)] pub struct MetastoreResolver { per_backend_factories: Arc>, @@ -55,7 +56,10 @@ impl MetastoreResolver { } /// Resolves the given `uri`. - pub async fn resolve(&self, uri: &Uri) -> Result, MetastoreResolverError> { + pub async fn resolve( + &self, + uri: &Uri, + ) -> Result { let backend = match uri.protocol() { Protocol::Azure => MetastoreBackend::File, Protocol::File => MetastoreBackend::File, diff --git a/quickwit/quickwit-metastore/src/metrics.rs b/quickwit/quickwit-metastore/src/metrics.rs deleted file mode 100644 index 1481b01fb1d..00000000000 --- a/quickwit/quickwit-metastore/src/metrics.rs +++ /dev/null @@ -1,56 +0,0 @@ -// 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 once_cell::sync::Lazy; -use quickwit_common::metrics::{new_counter_vec, new_histogram_vec, HistogramVec, IntCounterVec}; - -pub struct MetastoreMetrics { - pub requests_total: IntCounterVec<2>, - pub request_errors_total: IntCounterVec<2>, - pub request_duration_seconds: HistogramVec<3>, -} - -impl Default for MetastoreMetrics { - fn default() -> Self { - Self { - requests_total: new_counter_vec( - "requests_total", - "Number of requests", - "quickwit_metastore", - ["operation", "index"], - ), - request_errors_total: new_counter_vec( - "request_errors_total", - "Number of failed requests", - "quickwit_metastore", - ["operation", "index"], - ), - request_duration_seconds: new_histogram_vec( - "request_duration_seconds", - "Duration of requests", - "quickwit_metastore", - ["operation", "index", "error"], - ), - } - } -} - -/// `METASTORE_METRICS` exposes a bunch of metastore-related metrics through a Prometheus -/// endpoint. -pub static METASTORE_METRICS: Lazy = Lazy::new(MetastoreMetrics::default); diff --git a/quickwit/quickwit-metastore/src/tests.rs b/quickwit/quickwit-metastore/src/tests.rs index 8bf86dfefa3..8d9d2c8b68c 100644 --- a/quickwit/quickwit-metastore/src/tests.rs +++ b/quickwit/quickwit-metastore/src/tests.rs @@ -21,7 +21,6 @@ pub mod test_suite { use std::collections::BTreeSet; use std::num::NonZeroUsize; - use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; @@ -30,7 +29,14 @@ pub mod test_suite { 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::{DeleteQuery, EntityKind, MetastoreError, SourceType}; + 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, + }; use quickwit_proto::IndexUid; use quickwit_query::query_ast::qast_json_helper; use time::OffsetDateTime; @@ -41,8 +47,10 @@ pub mod test_suite { IndexCheckpointDelta, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, }; use crate::{ - ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitMaturity, SplitMetadata, - SplitState, + AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, + ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, ListSplitsQuery, + ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, Split, SplitMaturity, + SplitMetadata, SplitState, StageSplitsRequestExt, }; #[async_trait] @@ -62,28 +70,46 @@ pub mod test_suite { tags.iter().map(|tag| tag.to_string()).collect() } - async fn cleanup_index(metastore: &dyn Metastore, index_uid: IndexUid) { + async fn cleanup_index(metastore: &mut dyn MetastoreService, index_uid: IndexUid) { // List all splits. - let all_splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); + 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<&str> = - all_splits.iter().map(|split| split.split_id()).collect(); + 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.to_string(), all_split_ids.clone()); metastore - .mark_splits_for_deletion(index_uid.clone(), &all_split_ids) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); // Delete splits. + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: all_split_ids, + }; metastore - .delete_splits(index_uid.clone(), &all_split_ids) + .delete_splits(delete_splits_request) .await .unwrap(); } // Delete index. - metastore.delete_index(index_uid).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); } // Index API tests @@ -94,32 +120,49 @@ pub mod test_suite { // - list_indexes // - delete_index - pub async fn test_metastore_create_index() { - let metastore = MetastoreToTest::default_for_test().await; + 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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(&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(index_config).await.unwrap_err(); + let error = metastore + .create_index(create_index_request) + .await + .unwrap_err(); assert!(matches!(error, MetastoreError::AlreadyExists { .. })); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } pub async fn test_metastore_create_index_with_maximum_length< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + let mut metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix(format!("very-long-index-{}", "a".repeat(233)).as_str()); @@ -128,38 +171,56 @@ pub mod test_suite { let index_config = IndexConfig::for_test(&index_id, &index_uri); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_index_exists() { - let metastore = MetastoreToTest::default_for_test().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(index_config).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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_index_metadata() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_index_metadata< + MetastoreToTest: MetastoreService + 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("index-not-found") + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) .await .unwrap_err(); assert!(matches!( @@ -167,18 +228,32 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(&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); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_list_all_indexes() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_list_all_indexes< + MetastoreToTest: MetastoreService + 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"); @@ -188,34 +263,47 @@ pub mod test_suite { 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_metadatas(ListIndexesQuery::All) + .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(index_config_1).await.unwrap(); - let index_uid_2 = metastore.create_index(index_config_2).await.unwrap(); + 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_metadatas(ListIndexesQuery::All) + .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(&metastore, index_uid_1).await; - cleanup_index(&metastore, index_uid_2).await; + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; } - pub async fn test_metastore_list_indexes() { - let metastore = MetastoreToTest::default_for_test().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"); @@ -234,45 +322,81 @@ pub mod test_suite { let index_uri_4 = format!("ram:///indexes/{index_id_4}"); let index_config_4 = IndexConfig::for_test(&index_id_4, &index_uri_4); + let list_index_metadata_query = crate::ListIndexesQuery::IndexIdPatterns(vec![ + format!("prefix-*-{index_id_fragment}-suffix-*"), + format!("prefix*{index_id_fragment}*suffix-*"), + ]); let indexes_count = metastore - .list_indexes_metadatas(crate::ListIndexesQuery::IndexIdPatterns(vec![ - format!("prefix-*-{index_id_fragment}-suffix-*"), - format!("prefix*{index_id_fragment}*suffix-*"), - ])) + .list_indexes_metadata( + ListIndexesMetadataRequest::try_from_list_indexes_query(list_index_metadata_query) + .unwrap(), + ) .await .unwrap() + .deserialize_indexes_metadata() + .unwrap() .len(); assert_eq!(indexes_count, 0); - let index_uid_1 = metastore.create_index(index_config_1).await.unwrap(); - let index_uid_2 = metastore.create_index(index_config_2).await.unwrap(); - let index_uid_3 = metastore.create_index(index_config_3).await.unwrap(); - let index_uid_4 = metastore.create_index(index_config_4).await.unwrap(); + 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 list_indexes_query = crate::ListIndexesQuery::IndexIdPatterns(vec![format!( + "prefix-*-{index_id_fragment}-suffix-*" + )]); let indexes_count = metastore - .list_indexes_metadatas(crate::ListIndexesQuery::IndexIdPatterns(vec![format!( - "prefix-*-{index_id_fragment}-suffix-*" - )])) + .list_indexes_metadata( + ListIndexesMetadataRequest::try_from_list_indexes_query(list_indexes_query) + .unwrap(), + ) .await .unwrap() + .deserialize_indexes_metadata() + .unwrap() .len(); assert_eq!(indexes_count, 2); - cleanup_index(&metastore, index_uid_1).await; - cleanup_index(&metastore, index_uid_2).await; - cleanup_index(&metastore, index_uid_3).await; - cleanup_index(&metastore, index_uid_4).await; + 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() { - let metastore = MetastoreToTest::default_for_test().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 error = metastore - .delete_index(IndexUid::new("index-not-found")) + .delete_index(DeleteIndexRequest { + index_uid: "index-not-found".to_string(), + }) .await .unwrap_err(); assert!(matches!( @@ -281,7 +405,9 @@ pub mod test_suite { )); let error = metastore - .delete_index(IndexUid::new("test-delete-index")) + .delete_index(DeleteIndexRequest { + index_uid: "test-delete-index".to_string(), + }) .await .unwrap_err(); assert!(matches!( @@ -289,9 +415,21 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(index_uid.clone()).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); assert!(!metastore.index_exists(&index_id).await.unwrap()); @@ -302,12 +440,17 @@ pub mod test_suite { ..Default::default() }; - let index_uid = metastore.create_index(index_config).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) .await - .unwrap(); + .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. @@ -316,17 +459,24 @@ pub mod test_suite { // let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); // assert_eq!(splits.len(), 1) - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_add_source() { - let metastore = MetastoreToTest::default_for_test().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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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"); @@ -342,21 +492,27 @@ pub mod test_suite { assert_eq!( metastore - .index_metadata(&index_id) + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) .await .unwrap() + .deserialize_index_metadata() + .unwrap() .checkpoint .source_checkpoint(&source_id), None ); - metastore - .add_source(index_uid.clone(), source.clone()) + 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 index_metadata = metastore.index_metadata(&index_id).await.unwrap(); - let sources = &index_metadata.sources; assert_eq!(sources.len(), 1); assert!(sources.contains_key(&source_id)); @@ -372,36 +528,57 @@ pub mod test_suite { assert!(matches!( metastore - .add_source(index_uid.clone(), source.clone()) + .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(IndexUid::new("index-not-found"), source.clone()) + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new("index-not-found"), + source.clone() + ) + .unwrap() + ) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); assert!(matches!( metastore - .add_source(IndexUid::new(index_id), source) + .add_source( + AddSourceRequest::try_from_source_config(IndexUid::new(index_id), source) + .unwrap() + ) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_toggle_source() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_toggle_source< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + 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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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 { @@ -413,37 +590,61 @@ pub mod test_suite { transform_config: None, input_format: SourceInputFormat::Json, }; - metastore - .add_source(index_uid.clone(), source.clone()) + 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 index_metadata = metastore.index_metadata(&index_id).await.unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, true); // Disable source. metastore - .toggle_source(index_uid.clone(), &source.source_id, false) + .toggle_source(ToggleSourceRequest { + index_uid: index_uid.to_string(), + 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 index_metadata = metastore.index_metadata(&index_id).await.unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, false); // Enable source. metastore - .toggle_source(index_uid.clone(), &source.source_id, true) + .toggle_source(ToggleSourceRequest { + index_uid: index_uid.to_string(), + 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 index_metadata = metastore.index_metadata(&index_id).await.unwrap(); let source = index_metadata.sources.get(&source_id).unwrap(); assert_eq!(source.enabled, true); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_delete_source() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_delete_source< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + 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}"); @@ -461,67 +662,116 @@ pub mod test_suite { let index_config = IndexConfig::for_test(&index_id, index_uri.as_str()); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(IndexUid::new("index-not-found"), source.clone()) + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new("index-not-found"), + source.clone() + ) + .unwrap() + ) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); assert!(matches!( metastore - .add_source(IndexUid::new(&index_id), source.clone()) + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new(&index_id), + source.clone() + ) + .unwrap() + ) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); metastore - .add_source(index_uid.clone(), source) + .add_source( + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) + .unwrap(), + ) .await .unwrap(); metastore - .delete_source(index_uid.clone(), &source_id) + .delete_source(DeleteSourceRequest { + index_uid: index_uid.to_string(), + source_id: source_id.clone(), + }) .await .unwrap(); - let sources = metastore.index_metadata(&index_id).await.unwrap().sources; + 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(index_uid.clone(), &source_id) + .delete_source(DeleteSourceRequest { + index_uid: index_uid.to_string(), + source_id: source_id.to_string() + }) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Source { .. }) )); assert!(matches!( metastore - .delete_source(IndexUid::new("index-not-found"), &source_id) + .delete_source(DeleteSourceRequest { + index_uid: IndexUid::new("index-not-found").to_string(), + source_id: source_id.to_string() + }) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); assert!(matches!( metastore - .delete_source(IndexUid::new(index_id), &source_id) + .delete_source(DeleteSourceRequest { + index_uid: IndexUid::new(index_id).to_string(), + source_id: source_id.to_string() + }) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_reset_checkpoint() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_reset_checkpoint< + MetastoreToTest: MetastoreService + 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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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(); @@ -537,7 +787,10 @@ pub mod test_suite { input_format: SourceInputFormat::Json, }; metastore - .add_source(index_uid.clone(), source.clone()) + .add_source( + AddSourceRequest::try_from_source_config(index_uid.clone(), source.clone()) + .unwrap(), + ) .await .unwrap(); @@ -546,28 +799,43 @@ pub mod test_suite { 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 - .stage_splits(index_uid.clone(), vec![split_metadata]) - .await - .unwrap(); - metastore - .publish_splits(index_uid.clone(), &[split_id], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap(); } assert!(!metastore - .index_metadata(&index_id) + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) .await .unwrap() + .deserialize_index_metadata() + .unwrap() .checkpoint .is_empty()); metastore - .reset_source_checkpoint(index_uid.clone(), &source_ids[0]) + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: index_uid.to_string(), + source_id: source_ids[0].clone(), + }) .await .unwrap(); - let index_metadata = metastore.index_metadata(&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!(index_metadata .checkpoint .source_checkpoint(&source_ids[0]) @@ -580,7 +848,10 @@ pub mod test_suite { assert!(matches!( metastore - .reset_source_checkpoint(IndexUid::new("index-not-found"), &source_ids[1]) + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: IndexUid::new("index-not-found").to_string(), + source_id: source_ids[1].clone(), + }) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) @@ -588,31 +859,39 @@ pub mod test_suite { assert!(matches!( metastore - .reset_source_checkpoint(IndexUid::new(&index_id), &source_ids[1]) + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: IndexUid::new(&index_id).to_string(), + source_id: source_ids[1].to_string(), + }) .await .unwrap_err(), MetastoreError::NotFound(EntityKind::Index { .. }) )); metastore - .reset_source_checkpoint(index_uid.clone(), &source_ids[1]) + .reset_source_checkpoint(ResetSourceCheckpointRequest { + index_uid: index_uid.to_string(), + source_id: source_ids[1].to_string(), + }) .await .unwrap(); assert!(metastore - .index_metadata(&index_id) + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) .await .unwrap() + .deserialize_index_metadata() + .unwrap() .checkpoint .is_empty()); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } pub async fn test_metastore_publish_splits_empty_splits_array_is_allowed< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + let mut metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix("test-publish-splits-empty"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -621,18 +900,17 @@ pub mod test_suite { // Publish a split on a non-existent index { + let publish_splits_request = PublishSplitsRequest { + index_uid: index_id.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( - IndexUid::new("index-not-found"), - &[], - &[], - { - let offsets = 1..10; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -645,24 +923,34 @@ pub mod test_suite { // checkpoint. This operation is allowed and used in the Indexer. { let index_config = IndexConfig::for_test(&index_id, &index_uri); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - + 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( - index_uid.clone(), - &[], - &[], - { - let offsets = 0..100; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); - let index_metadata = metastore.index_metadata(&index_id).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) @@ -674,12 +962,14 @@ pub mod test_suite { .unwrap(), &Position::from(100u64 - 1) ); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } - pub async fn test_metastore_publish_splits() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_publish_splits< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + let mut metastore = MetastoreToTest::default_for_test().await; let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); @@ -710,18 +1000,18 @@ pub mod test_suite { // Publish a split on a non-existent index { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new("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( - IndexUid::new("index-not-found"), - &["split-not-found"], - &[], - { - let offsets = 1..10; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -732,18 +1022,18 @@ pub mod test_suite { // Publish a split on a wrong index uid { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new(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( - IndexUid::new(index_id), - &["split-not-found"], - &[], - { - let offsets = 1..10; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -754,10 +1044,21 @@ pub mod test_suite { // Publish a non-existent split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - + 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.to_string(), + staged_split_ids: vec!["split-not-found".to_string()], + ..Default::default() + }; let error = metastore - .publish_splits(index_uid.clone(), &["split-not-found"], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -765,116 +1066,146 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a staged split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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.to_string(), + staged_split_ids: vec![split_id_1.clone()], + ..Default::default() + }; metastore - .publish_splits(index_uid.clone(), &[&split_id_1], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap(); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a published split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 1..12; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 1..12; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); + println!("{:?}", error); assert!(matches!( error, MetastoreError::FailedPrecondition { - entity: EntityKind::CheckpointDelta { .. }, + entity: EntityKind::Splits { .. }, .. } )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a non-staged split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 12..15; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id_1.clone()]); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_1]) + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 15..18; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -885,30 +1216,39 @@ pub mod test_suite { } )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a staged split and non-existent split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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( - index_uid.clone(), - &[&split_id_1, "split-not-found"], - &[], - { - let offsets = 15..18; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -916,45 +1256,54 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a published split and non-existent split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 15..18; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .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( - index_uid.clone(), - &[&split_id_1, "split-not-found"], - &[], - { - let offsets = 18..24; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -962,50 +1311,61 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a non-staged split and non-existent split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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( - index_uid.clone(), - &[&split_id_1], - &[], - { - let offsets = 18..24; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id_1.clone()]); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_1]) + .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( - index_uid.clone(), - &[&split_id_1, "split-not-found"], - &[], - { - let offsets = 24..26; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1013,80 +1373,97 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish staged splits on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .unwrap() + .index_uid + .into(); - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_2.clone()]) - .await - .unwrap(); + 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( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - { - let offsets = 24..26; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish a staged split and published split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) + 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(); + .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( - index_uid.clone(), - &[&split_id_2], - &[], - { - let offsets = 26..28; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .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( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - { - let offsets = 28..30; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1097,48 +1474,54 @@ pub mod test_suite { } )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Publish published splits on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) + 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(); + .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( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - { - let offsets = 30..31; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .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( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - { - let offsets = 30..31; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_resquest) .await .unwrap_err(); assert!(matches!( @@ -1149,27 +1532,34 @@ pub mod test_suite { } )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } pub async fn test_metastore_publish_splits_concurrency< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest + Clone, >() { - let metastore: Arc = Arc::new(MetastoreToTest::default_for_test().await); + 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 = metastore.create_index(index_config).await.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 metastore = metastore.clone(); + let mut metastore_clone = metastore.clone(); let index_id = index_id.clone(); let source_id = source_id.clone(); @@ -1182,8 +1572,13 @@ pub mod test_suite { index_uid: index_uid.clone(), ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata]) + 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( @@ -1196,14 +1591,16 @@ pub mod test_suite { source_id, source_delta, }; - metastore - .publish_splits( - index_uid.clone(), - &[&split_id], - &[], - Some(checkpoint_delta), - None, - ) + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + 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(); } @@ -1212,7 +1609,12 @@ pub mod test_suite { } try_join_all(join_handles).await.unwrap(); - let index_metadata = metastore.index_metadata(&index_id).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) @@ -1220,11 +1622,13 @@ pub mod test_suite { assert_eq!(source_checkpoint.num_partitions(), 10); - cleanup_index(metastore.as_ref(), index_uid).await + cleanup_index(&mut metastore, index_uid).await } - pub async fn test_metastore_replace_splits() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_replace_splits< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + let mut metastore = MetastoreToTest::default_for_test().await; let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); @@ -1262,14 +1666,14 @@ pub mod test_suite { // Replace splits on a non-existent index { + let publish_splits_request = PublishSplitsRequest { + index_uid: IndexUid::new("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( - IndexUid::new("index-not-found"), - &["split-not-found-1"], - &["split-not-found-2"], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1280,17 +1684,24 @@ pub mod test_suite { // Replace a non-existent split on an index { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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.to_string(), + 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( - index_uid.clone(), - &["split-not-found-1"], - &["split-not-found-2"], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1298,32 +1709,46 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Replace a publish split with a non existing split { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .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(index_uid.clone(), &[&split_id_1], &[], None, None) + .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( - index_uid.clone(), - &[&split_id_2], - &[&split_id_1], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1331,46 +1756,53 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Replace a publish split with a deleted split { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) + 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(); + .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( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id_2.clone()]); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_2]) + .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( - index_uid.clone(), - &[&split_id_2], - &[&split_id_1], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap_err(); assert!(matches!( @@ -1381,36 +1813,52 @@ pub mod test_suite { } )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Replace a publish split with mixed splits { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .unwrap() + .index_uid + .into(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_1], &[], None, None) - .await - .unwrap(); + 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 - .stage_splits(index_uid.clone(), vec![split_metadata_2.clone()]) + .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( - index_uid.clone(), - &[&split_id_2, &split_id_3], - &[&split_id_1], - None, - None, - ) // TODO source id + .publish_splits(publish_splits_request) // TODO source id .await .unwrap_err(); assert!(matches!( @@ -1418,103 +1866,143 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Splits { .. }) )); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Replace a deleted split with a new split { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .unwrap() + .index_uid + .into(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_1], &[], None, None) - .await - .unwrap(); + 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 - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_1]) + .publish_splits(publish_splits_request) .await .unwrap(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id_1.clone()]); metastore - .stage_splits(index_uid.clone(), vec![split_metadata_2.clone()]) + .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( - index_uid.clone(), - &[&split_id_2], - &[&split_id_1], - None, - None, - ) + .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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } // Replace a publish split with staged splits { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1.clone()]) + 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(); + .unwrap() + .index_uid + .into(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_1], &[], None, None) - .await - .unwrap(); + 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 - .stage_splits( - index_uid.clone(), - vec![split_metadata_2.clone(), split_metadata_3.clone()], - ) + .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( - index_uid.clone(), - &[&split_id_2, &split_id_3], - &[&split_id_1], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } pub async fn test_metastore_mark_splits_for_deletion< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + 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 = metastore.create_index(index_config).await.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".to_string(), Vec::new()); let error = metastore - .mark_splits_for_deletion(IndexUid::new("index-not-found"), &[]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap_err(); assert!(matches!( @@ -1522,8 +2010,12 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); + let mark_splits_for_deletion_request = MarkSplitsForDeletionRequest::new( + index_uid.to_string(), + vec!["split-not-found".to_string()], + ); metastore - .mark_splits_for_deletion(index_uid.clone(), &["split-not-found"]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); @@ -1534,10 +2026,12 @@ pub mod test_suite { create_timestamp: current_timestamp, ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_1]) - .await - .unwrap(); + 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 { @@ -1546,12 +2040,19 @@ pub mod test_suite { 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 - .stage_splits(index_uid.clone(), vec![split_metadata_2]) - .await - .unwrap(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_2], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap(); @@ -1562,29 +2063,44 @@ pub mod test_suite { 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 - .stage_splits(index_uid.clone(), vec![split_metadata_3]) - .await - .unwrap(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_3], &[], None, None) + .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().to_string(), + vec![split_id_3.clone()], + ); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_3]) + .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( - ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::MarkedForDeletion), - ) + .list_splits(list_splits_request) .await + .unwrap() + .deserialize_splits() .unwrap(); assert_eq!(marked_splits.len(), 1); @@ -1596,20 +2112,30 @@ pub mod test_suite { // 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.to_string(), + vec![ + split_id_1.clone(), + split_id_2.clone(), + split_id_3.clone(), + "split-not-found".to_string(), + ], + ); metastore - .mark_splits_for_deletion( - index_uid.clone(), - &[&split_id_1, &split_id_2, &split_id_3, "split-not-found"], - ) + .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( - ListSplitsQuery::for_index(index_uid.clone()) - .with_split_state(SplitState::MarkedForDeletion), - ) + .list_splits(list_splits_request) .await + .unwrap() + .deserialize_splits() .unwrap(); marked_splits.sort_by_key(|split| split.split_id().to_string()); @@ -1625,20 +2151,33 @@ pub mod test_suite { assert_eq!(marked_splits[2].split_id(), split_id_3); assert_eq!(marked_splits[2].update_timestamp, split_3_update_timestamp); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_delete_splits() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_delete_splits< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + 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 index_uid = metastore.create_index(index_config).await.unwrap(); + 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("index-not-found").to_string(), + split_ids: Vec::new(), + }; let error = metastore - .delete_splits(IndexUid::new("index-not-found"), &[]) + .delete_splits(delete_splits_request) .await .unwrap_err(); @@ -1647,8 +2186,13 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); + // Check error if index does not exist. + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_id.to_string(), + split_ids: Vec::new(), + }; let error = metastore - .delete_splits(IndexUid::new(&index_id), &[]) + .delete_splits(delete_splits_request) .await .unwrap_err(); @@ -1657,8 +2201,12 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: vec!["split-not-found".to_string()], + }; metastore - .delete_splits(index_uid.clone(), &["split-not-found"]) + .delete_splits(delete_splits_request) .await .unwrap(); @@ -1668,12 +2216,19 @@ pub mod test_suite { 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 - .stage_splits(index_uid.clone(), vec![split_metadata_1]) - .await - .unwrap(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_1], &[], None, None) + .publish_splits(publish_splits_request) .await .unwrap(); @@ -1683,13 +2238,19 @@ pub mod test_suite { index_uid: index_uid.clone(), ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_2]) - .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 delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: vec![split_id_1.clone(), split_id_2.clone()], + }; let error = metastore - .delete_splits(index_uid.clone(), &[&split_id_1, &split_id_2]) + .delete_splits(delete_splits_request) .await .unwrap_err(); @@ -1703,40 +2264,55 @@ pub mod test_suite { assert_eq!( metastore - .list_all_splits(index_uid.clone()) + .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.to_string(), + vec![split_id_1.clone(), split_id_2.clone()], + ); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_1, &split_id_2]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); + let delete_splits_request = DeleteSplitsRequest { + index_uid: index_uid.to_string(), + split_ids: vec![ + split_id_1.clone(), + split_id_2.clone(), + "split-not-found".to_string(), + ], + }; metastore - .delete_splits( - index_uid.clone(), - &[&split_id_1, &split_id_2, "split-not-found"], - ) + .delete_splits(delete_splits_request) .await .unwrap(); assert_eq!( metastore - .list_all_splits(index_uid.clone()) + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await .unwrap() + .deserialize_splits() + .unwrap() .len(), 0 ); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_list_all_splits() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_list_all_splits< + MetastoreToTest: MetastoreService + DefaultForTest, + >() { + let mut metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix("test-list-all-splits"); let index_uid = IndexUid::new(index_id.clone()); @@ -1781,7 +2357,7 @@ pub mod test_suite { }; let error = metastore - .list_all_splits(IndexUid::new("index-not-found")) + .list_splits(ListSplitsRequest::try_from_index_uid("index-not-found".into()).unwrap()) .await .unwrap_err(); assert!(matches!( @@ -1790,40 +2366,53 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. } | EntityKind::Indexes { .. }) )); - let index_uid = metastore.create_index(index_config).await.unwrap(); - + 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 - .stage_splits( - index_uid.clone(), - vec![ - split_metadata_1, - split_metadata_2, - split_metadata_3, - split_metadata_4, - split_metadata_5, - split_metadata_6, - ], - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let mark_splits_for_deletion = MarkSplitsForDeletionRequest::new( + index_uid.clone().to_string(), + vec![split_id_3.clone(), split_id_4.clone()], + ); metastore - .publish_splits( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - None, - None, - ) + .mark_splits_for_deletion(mark_splits_for_deletion) .await .unwrap(); - metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_3, &split_id_4]) + let splits = metastore + .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await + .unwrap() + .deserialize_splits() .unwrap(); - - let splits = metastore.list_all_splits(index_uid.clone()).await.unwrap(); let split_ids = collect_split_ids(&splits); assert_eq!( split_ids, @@ -1837,11 +2426,11 @@ pub mod test_suite { ] ); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid.clone()).await; } - pub async fn test_metastore_list_splits() { - let metastore = MetastoreToTest::default_for_test().await; + pub async fn test_metastore_list_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); @@ -1916,7 +2505,10 @@ pub mod test_suite { { let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let error = metastore.list_splits(query).await.unwrap_err(); + 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. @@ -1924,24 +2516,35 @@ pub mod test_suite { )); } { - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - split_metadata_4.clone(), - split_metadata_5.clone(), - ], - ) + 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(); + .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(query).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!( splits.len(), 3, @@ -1949,7 +2552,12 @@ pub mod test_suite { ); let query = ListSplitsQuery::for_index(index_uid.clone()).with_offset(3); - let splits = metastore.list_splits(query).await.unwrap(); + let splits = metastore + .list_splits(ListSplitsRequest::try_from_list_splits_query(query).unwrap()) + .await + .unwrap() + .deserialize_splits() + .unwrap(); assert_eq!( splits.len(), 2, @@ -1961,7 +2569,12 @@ pub mod test_suite { .with_time_range_start_gte(0) .with_time_range_end_lt(99); - let splits = metastore.list_splits(query).await.unwrap(); + 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()) @@ -1973,14 +2586,24 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(200); - let splits = metastore.list_splits(query).await.unwrap(); + 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(query).await.unwrap(); + 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]); @@ -1988,7 +2611,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(100); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -1996,7 +2624,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(101); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2004,7 +2637,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(199); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2012,7 +2650,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(200); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2020,7 +2663,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(201); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2031,7 +2679,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(299); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2042,7 +2695,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(300); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2053,7 +2711,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(0) .with_time_range_end_lt(301); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2070,7 +2733,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(301) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2078,7 +2746,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(300) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2086,7 +2759,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(299) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2094,7 +2772,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(201) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2102,7 +2785,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(200) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2110,7 +2798,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(199) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2121,7 +2814,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(101) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2132,7 +2830,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(101) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2143,7 +2846,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(100) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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!( @@ -2155,7 +2863,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(99) .with_time_range_end_lt(400); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2172,7 +2885,12 @@ pub mod test_suite { .with_split_state(SplitState::Staged) .with_time_range_start_gte(1000) .with_time_range_end_lt(1100); - let splits = metastore.list_splits(query).await.unwrap(); + 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]); @@ -2187,14 +2905,21 @@ pub mod test_suite { create_timestamp: OffsetDateTime::now_utc().unix_timestamp(), ..Default::default() }; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata_6.clone()]) - .await - .unwrap(); + 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(query).await.unwrap(); + 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, @@ -2215,7 +2940,12 @@ pub mod test_suite { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_state(SplitState::Staged) .with_tags_filter(tag_filter_ast); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2230,7 +2960,12 @@ pub mod test_suite { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_update_timestamp_gte(current_timestamp); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2246,7 +2981,12 @@ pub mod test_suite { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_update_timestamp_gte(split_metadata_6.create_timestamp); - let splits = metastore.list_splits(query).await.unwrap(); + 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) @@ -2256,7 +2996,12 @@ pub mod test_suite { let query = ListSplitsQuery::for_index(index_uid.clone()) .with_create_timestamp_lt(split_metadata_6.create_timestamp); - let splits = metastore.list_splits(query).await.unwrap(); + 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, @@ -2270,7 +3015,12 @@ pub mod test_suite { ); let query = ListSplitsQuery::for_index(index_uid.clone()).with_delete_opstamp_lt(6); - let splits = metastore.list_splits(query.clone()).await.unwrap(); + 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, @@ -2282,7 +3032,12 @@ pub mod test_suite { 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(query.clone()).await.unwrap(); + 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, @@ -2291,18 +3046,23 @@ pub mod test_suite { let query = ListSplitsQuery::for_index(index_uid.clone()) .retain_immature(maturity_evaluation_timestamp); - let splits = metastore.list_splits(query.clone()).await.unwrap(); + 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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } pub async fn test_metastore_split_update_timestamp< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + let mut metastore = MetastoreToTest::default_for_test().await; let mut current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); @@ -2322,17 +3082,32 @@ pub mod test_suite { }; // Create an index - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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; - metastore - .stage_splits(index_uid.clone(), vec![split_metadata.clone()]) - .await - .unwrap(); + 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_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + 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()); @@ -2340,21 +3115,27 @@ pub mod test_suite { // wait for 1s, publish split & check `update_timestamp` sleep(Duration::from_secs(1)).await; + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + 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( - index_uid.clone(), - &[&split_id], - &[], - { - let offsets = 0..5; - IndexCheckpointDelta::for_test(&source_id, offsets) - } - .into(), - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); - let split_meta = metastore.list_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + 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, @@ -2364,23 +3145,40 @@ pub mod test_suite { // 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.to_string(), vec![split_id.clone()]); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id]) + .mark_splits_for_deletion(mark_splits_for_deletion_request) .await .unwrap(); - let split_meta = metastore.list_all_splits(index_uid.clone()).await.unwrap()[0].clone(); + 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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_create_delete_task() { - let metastore = MetastoreToTest::default_for_test().await; + 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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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.to_string(), query_ast: qast_json_helper("my_field:my_value", &[]), @@ -2430,25 +3228,35 @@ pub mod test_suite { .unwrap(); assert!(delete_task_2.opstamp > delete_task_1.opstamp); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } - pub async fn test_metastore_last_delete_opstamp() { - let metastore = MetastoreToTest::default_for_test().await; + 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 = metastore - .create_index(index_config_1.clone()) + let index_uid_1: IndexUid = metastore + .create_index( + CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap(), + ) .await - .unwrap(); - let index_uid_2 = metastore - .create_index(index_config_2.clone()) + .unwrap() + .index_uid + .into(); + let index_uid_2: IndexUid = metastore + .create_index( + CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap(), + ) .await - .unwrap(); + .unwrap() + .index_uid + .into(); let delete_query_index_1 = DeleteQuery { index_uid: index_uid_1.to_string(), @@ -2464,9 +3272,12 @@ pub mod test_suite { }; let last_opstamp_index_1_with_no_task = metastore - .last_delete_opstamp(index_uid_1.clone()) + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_1.to_string(), + }) .await - .unwrap(); + .unwrap() + .last_delete_opstamp; assert_eq!(last_opstamp_index_1_with_no_task, 0); // Create a delete task. @@ -2484,27 +3295,40 @@ pub mod test_suite { .unwrap(); let last_opstamp_index_1 = metastore - .last_delete_opstamp(index_uid_1.clone()) + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_1.to_string(), + }) .await - .unwrap(); + .unwrap() + .last_delete_opstamp; let last_opstamp_index_2 = metastore - .last_delete_opstamp(index_uid_2.clone()) + .last_delete_opstamp(LastDeleteOpstampRequest { + index_uid: index_uid_2.to_string(), + }) .await - .unwrap(); + .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(&metastore, index_uid_1).await; - cleanup_index(&metastore, index_uid_2).await; + 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: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + 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 index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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.to_string(), query_ast: qast_json_helper("my_field:my_value", &[]), @@ -2520,25 +3344,40 @@ pub mod test_suite { .await .unwrap(); - metastore.delete_index(index_uid).await.unwrap(); + metastore + .delete_index(DeleteIndexRequest { + index_uid: index_uid.to_string(), + }) + .await + .unwrap(); } - pub async fn test_metastore_list_delete_tasks() { - let metastore = MetastoreToTest::default_for_test().await; + 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 = metastore - .create_index(index_config_1.clone()) + let index_uid_1: IndexUid = metastore + .create_index( + CreateIndexRequest::try_from_index_config(index_config_1.clone()).unwrap(), + ) .await - .unwrap(); - let index_uid_2 = metastore - .create_index(index_config_2.clone()) + .unwrap() + .index_uid + .into(); + let index_uid_2: IndexUid = metastore + .create_index( + CreateIndexRequest::try_from_index_config(index_config_2.clone()).unwrap(), + ) .await - .unwrap(); + .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", &[]), @@ -2567,26 +3406,33 @@ pub mod test_suite { .unwrap(); let all_index_id_1_delete_tasks = metastore - .list_delete_tasks(index_uid_1.clone(), 0) + .list_delete_tasks(ListDeleteTasksRequest::new(index_uid_1.to_string(), 0)) .await - .unwrap(); + .unwrap() + .delete_tasks; assert_eq!(all_index_id_1_delete_tasks.len(), 2); let recent_index_id_1_delete_tasks = metastore - .list_delete_tasks(index_uid_1.clone(), delete_task_1.opstamp) + .list_delete_tasks(ListDeleteTasksRequest::new( + index_uid_1.to_string(), + delete_task_1.opstamp, + )) .await - .unwrap(); + .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(&metastore, index_uid_1).await; - cleanup_index(&metastore, index_uid_2).await; + cleanup_index(&mut metastore, index_uid_1).await; + cleanup_index(&mut metastore, index_uid_2).await; } - pub async fn test_metastore_list_stale_splits() { - let metastore = MetastoreToTest::default_for_test().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(index_id.clone()); @@ -2638,8 +3484,13 @@ pub mod test_suite { ..Default::default() }; + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: IndexUid::new("index-not-found").to_string(), + delete_opstamp: 0, + num_splits: 100, + }; let error = metastore - .list_stale_splits(IndexUid::new("index-not-found"), 0, 10) + .list_stale_splits(list_stale_splits_request) .await .unwrap_err(); assert!(matches!( @@ -2649,47 +3500,66 @@ pub mod test_suite { { info!("List stale splits on an index"); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - split_metadata_5.clone(), - ], - ) + 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(); + .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.to_string(), + staged_split_ids: vec![split_id_4.clone()], + ..Default::default() + }; metastore - .stage_splits(index_uid.clone(), vec![split_metadata_4.clone()]) - .await - .unwrap(); - metastore - .publish_splits(index_uid.clone(), &[&split_id_4], &[], None, None) + .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.to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone(), split_id_5.clone()], + ..Default::default() + }; metastore - .publish_splits( - index_uid.clone(), - &[&split_id_1, &split_id_2, &split_id_5], - &[], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 100, + num_splits: 1, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 100, 1) + .list_stale_splits(list_stale_splits_request) .await + .unwrap() + .deserialize_splits() .unwrap(); assert_eq!(splits.len(), 1); assert_eq!( @@ -2697,9 +3567,16 @@ pub mod test_suite { split_metadata_2.delete_opstamp ); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 100, + num_splits: 4, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 100, 4) + .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()); @@ -2710,9 +3587,16 @@ pub mod test_suite { split_metadata_1.delete_opstamp ); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 20, + num_splits: 2, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 20, 2) + .list_stale_splits(list_stale_splits_request) .await + .unwrap() + .deserialize_splits() .unwrap(); assert_eq!(splits.len(), 1); assert_eq!( @@ -2720,19 +3604,26 @@ pub mod test_suite { split_metadata_2.delete_opstamp ); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 10, + num_splits: 2, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 10, 2) + .list_stale_splits(list_stale_splits_request) .await + .unwrap() + .deserialize_splits() .unwrap(); assert!(splits.is_empty()); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } pub async fn test_metastore_update_splits_delete_opstamp< - MetastoreToTest: Metastore + DefaultForTest, + MetastoreToTest: MetastoreService + DefaultForTest, >() { - let metastore = MetastoreToTest::default_for_test().await; + 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(index_id.clone()); @@ -2766,8 +3657,13 @@ pub mod test_suite { { info!("Update splits delete opstamp on a non-existent index."); + let update_splits_delete_opstamp_request = UpdateSplitsDeleteOpstampRequest { + index_uid: IndexUid::new("index-not-found").to_string(), + split_ids: vec![split_id_1.clone()], + delete_opstamp: 10, + }; let metastore_err = metastore - .update_splits_delete_opstamp(IndexUid::new("index-not-found"), &[&split_id_1], 10) + .update_splits_delete_opstamp(update_splits_delete_opstamp_request) .await .unwrap_err(); error!(err=?metastore_err); @@ -2779,61 +3675,92 @@ pub mod test_suite { { info!("Update splits delete opstamp on an index."); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); - - metastore - .stage_splits( - index_uid.clone(), - vec![ - split_metadata_1.clone(), - split_metadata_2.clone(), - split_metadata_3.clone(), - ], - ) + 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(); + .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.to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; metastore - .publish_splits( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - None, - None, - ) + .publish_splits(publish_splits_request) .await .unwrap(); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 100, + num_splits: 2, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 100, 2) + .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.to_string(), + split_ids: vec![split_id_1.clone(), split_id_2.clone()], + delete_opstamp: 100, + }; metastore - .update_splits_delete_opstamp(index_uid.clone(), &[&split_id_1, &split_id_2], 100) + .update_splits_delete_opstamp(update_splits_delete_opstamp_request) .await .unwrap(); + let list_stale_splits_request = ListStaleSplitsRequest { + index_uid: index_uid.to_string(), + delete_opstamp: 100, + num_splits: 2, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 100, 2) + .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.to_string(), + delete_opstamp: 200, + num_splits: 2, + }; let splits = metastore - .list_stale_splits(index_uid.clone(), 200, 2) + .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(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } - pub async fn test_metastore_stage_splits() { - let metastore = MetastoreToTest::default_for_test().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(index_id.clone()); @@ -2858,11 +3785,13 @@ pub mod test_suite { }; // Stage a splits on a non-existent index + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + IndexUid::new("index-not-found"), + vec![split_metadata_1.clone()], + ) + .unwrap(); let error = metastore - .stage_splits( - IndexUid::new("index-not-found"), - vec![split_metadata_1.clone()], - ) + .stage_splits(stage_splits_request) .await .unwrap_err(); assert!(matches!( @@ -2870,41 +3799,61 @@ pub mod test_suite { MetastoreError::NotFound(EntityKind::Index { .. }) )); - let index_uid = metastore.create_index(index_config.clone()).await.unwrap(); + 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 - metastore - .stage_splits( - index_uid.clone(), - vec![split_metadata_1.clone(), split_metadata_2.clone()], - ) - .await - .unwrap(); + 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(query).await.unwrap(); + 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(index_uid.clone(), vec![split_metadata_1.clone()]) + .stage_splits(stage_splits_request) .await .expect("Pre-existing staged splits should be updated."); + let publish_splits_request = PublishSplitsRequest { + index_uid: index_uid.to_string(), + staged_split_ids: vec![split_id_1.clone(), split_id_2.clone()], + ..Default::default() + }; metastore - .publish_splits( - index_uid.clone(), - &[&split_id_1, &split_id_2], - &[], - None, - None, - ) + .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(index_uid.clone(), vec![split_metadata_1.clone()]) + .stage_splits(stage_splits_request) .await .expect_err( "Metastore should not allow splits which are not `Staged` to be overwritten.", @@ -2917,12 +3866,19 @@ pub mod test_suite { } ),); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), vec![split_id_2.clone()]); metastore - .mark_splits_for_deletion(index_uid.clone(), &[&split_id_2]) + .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(index_uid.clone(), vec![split_metadata_2.clone()]) + .stage_splits(stage_splits_request) .await .expect_err( "Metastore should not allow splits which are not `Staged` to be overwritten.", @@ -2935,7 +3891,7 @@ pub mod test_suite { } ),); - cleanup_index(&metastore, index_uid).await; + cleanup_index(&mut metastore, index_uid).await; } } diff --git a/quickwit/quickwit-opentelemetry/src/otlp/logs.rs b/quickwit/quickwit-opentelemetry/src/otlp/logs.rs index 64f7755032e..d2f6e604227 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/logs.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/logs.rs @@ -465,7 +465,8 @@ impl LogsService for OtlpGrpcLogsService { #[cfg(test)] mod tests { - use quickwit_metastore::metastore_for_test; + use quickwit_metastore::{metastore_for_test, CreateIndexRequestExt}; + use quickwit_proto::metastore::{CreateIndexRequest, MetastoreService}; use super::*; @@ -478,9 +479,10 @@ mod tests { #[tokio::test] async fn test_create_index() { - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_config = OtlpGrpcLogsService::index_config(&Uri::for_test("ram:///indexes")).unwrap(); - metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + metastore.create_index(create_index_request).await.unwrap(); } } diff --git a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs index edaa3ce08a0..7d959b1c981 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs @@ -799,7 +799,8 @@ impl TraceService for OtlpGrpcTracesService { #[cfg(test)] mod tests { - use quickwit_metastore::metastore_for_test; + use quickwit_metastore::{metastore_for_test, CreateIndexRequestExt}; + use quickwit_proto::metastore::{CreateIndexRequest, MetastoreService}; use quickwit_proto::opentelemetry::proto::common::v1::any_value::Value as OtlpAnyValueValue; use quickwit_proto::opentelemetry::proto::common::v1::{ AnyValue as OtlpAnyValue, KeyValue as OtlpKeyValue, @@ -820,10 +821,11 @@ mod tests { #[tokio::test] async fn test_create_index() { - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_config = OtlpGrpcTracesService::index_config(&Uri::for_test("ram:///indexes")).unwrap(); - metastore.create_index(index_config).await.unwrap(); + let create_index_request = CreateIndexRequest::try_from_index_config(index_config).unwrap(); + metastore.create_index(create_index_request).await.unwrap(); } #[test] diff --git a/quickwit/quickwit-proto/Cargo.toml b/quickwit/quickwit-proto/Cargo.toml index 9efa4db9fc9..dec5bd97a47 100644 --- a/quickwit/quickwit-proto/Cargo.toml +++ b/quickwit/quickwit-proto/Cargo.toml @@ -16,6 +16,7 @@ bytes = { workspace = true } dyn-clone = { workspace = true } http = { workspace = true } hyper = { workspace = true } +futures = { workspace = true, optional = true } mockall = { workspace = true, optional = true } opentelemetry = { workspace = true } prost = { workspace = true } @@ -37,6 +38,7 @@ quickwit-common = { workspace = true } [dev-dependencies] mockall = { workspace = true } +futures = { workspace = true } [build-dependencies] glob = "0.3.1" @@ -47,4 +49,4 @@ quickwit-codegen = { workspace = true } [features] postgres = [ "sqlx" ] -testsuite = [ "mockall" ] +testsuite = [ "mockall", "futures" ] diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index a8fc85ea3c6..6ed886a0534 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -23,66 +23,64 @@ use glob::glob; use quickwit_codegen::Codegen; fn main() -> Result<(), Box> { - // Prost + tonic + Quickwit codegen for control plane, indexing, and ingest services. + // Prost + tonic + Quickwit codegen for control plane, indexing, metastore, ingest and search + // services. // - // Control plane + // Control plane. Codegen::run( &["protos/quickwit/control_plane.proto"], "src/codegen/quickwit", "crate::control_plane::ControlPlaneResult", "crate::control_plane::ControlPlaneError", false, + false, &["protos"], ) .unwrap(); - // Indexing Service + // Indexing Service. Codegen::run( &["protos/quickwit/indexing.proto"], "src/codegen/quickwit", "crate::indexing::IndexingResult", "crate::indexing::IndexingError", false, + false, &[], ) .unwrap(); - // Ingest service - let mut prost_config = prost_build::Config::default(); - prost_config.bytes(["DocBatchV2.doc_buffer", "MRecordBatch.mrecord_buffer"]); + // Metastore service. + let mut metastore_api_config = prost_build::Config::default(); + metastore_api_config + .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") + .field_attribute("DeleteQuery.query_ast", "#[serde(alias = \"query\")]") + .field_attribute( + "DeleteQuery.start_timestamp", + "#[serde(skip_serializing_if = \"Option::is_none\")]", + ) + .field_attribute( + "DeleteQuery.end_timestamp", + "#[serde(skip_serializing_if = \"Option::is_none\")]", + ); Codegen::run_with_config( - &[ - "protos/quickwit/ingester.proto", - "protos/quickwit/router.proto", - ], + &["protos/quickwit/metastore.proto"], "src/codegen/quickwit", - "crate::ingest::IngestV2Result", - "crate::ingest::IngestV2Error", - false, + "crate::metastore::MetastoreResult", + "crate::metastore::MetastoreError", + true, + true, &["protos"], - prost_config, + metastore_api_config, ) .unwrap(); - // "Classic" prost + tonic codegen for metastore and search services. + // Ingest service (metastore service proto should be generated before ingest). let mut prost_config = prost_build::Config::default(); prost_config .bytes(["DocBatchV2.doc_buffer", "MRecordBatch.mrecord_buffer"]) - .protoc_arg("--experimental_allow_proto3_optional"); - - tonic_build::configure() - .enum_attribute(".", "#[serde(rename_all=\"snake_case\")]") - .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") - .field_attribute("DeleteQuery.query_ast", "#[serde(alias = \"query\")]") - .field_attribute( - "DeleteQuery.start_timestamp", - "#[serde(skip_serializing_if = \"Option::is_none\")]", - ) - .field_attribute( - "DeleteQuery.end_timestamp", - "#[serde(skip_serializing_if = \"Option::is_none\")]", - ) + .type_attribute("Shard", "#[derive(Eq)]") .field_attribute( "Shard.follower_id", "#[serde(default, skip_serializing_if = \"Option::is_none\")]", @@ -98,23 +96,37 @@ fn main() -> Result<(), Box> { .field_attribute( "Shard.replication_position_inclusive", "#[serde(default, skip_serializing_if = \"Option::is_none\")]", - ) + ); + + Codegen::run_with_config( + &[ + "protos/quickwit/ingester.proto", + "protos/quickwit/router.proto", + ], + "src/codegen/quickwit", + "crate::ingest::IngestV2Result", + "crate::ingest::IngestV2Error", + false, + false, + &["protos"], + prost_config, + ) + .unwrap(); + + // Search service. + let mut prost_config = prost_build::Config::default(); + prost_config.protoc_arg("--experimental_allow_proto3_optional"); + + tonic_build::configure() + .enum_attribute(".", "#[serde(rename_all=\"snake_case\")]") .type_attribute(".", "#[derive(Serialize, Deserialize, utoipa::ToSchema)]") .type_attribute("PartialHit", "#[derive(Eq, Hash)]") .type_attribute("PartialHit.sort_value", "#[derive(Copy)]") .type_attribute("SearchRequest", "#[derive(Eq, Hash)]") - .type_attribute("Shard", "#[derive(Eq)]") .type_attribute("SortByValue", "#[derive(Ord, PartialOrd)]") .type_attribute("SortField", "#[derive(Eq, Hash)]") .out_dir("src/codegen/quickwit") - .compile_with_config( - prost_config, - &[ - "protos/quickwit/metastore.proto", - "protos/quickwit/search.proto", - ], - &["protos"], - )?; + .compile_with_config(prost_config, &["protos/quickwit/search.proto"], &["protos"])?; // Jaeger proto let protos = find_protos("protos/third-party/jaeger"); diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index f7f33da2ed0..5824faaa9dd 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -40,61 +40,58 @@ enum SourceType { service MetastoreService { // Creates an index. - rpc create_index(CreateIndexRequest) returns (CreateIndexResponse); + rpc CreateIndex(CreateIndexRequest) returns (CreateIndexResponse); // Gets an index metadata. - rpc index_metadata(IndexMetadataRequest) returns (IndexMetadataResponse); + rpc IndexMetadata(IndexMetadataRequest) returns (IndexMetadataResponse); // Gets an indexes metadatas. - rpc list_indexes_metadatas(ListIndexesMetadatasRequest) returns (ListIndexesMetadatasResponse); + rpc ListIndexesMetadata(ListIndexesMetadataRequest) returns (ListIndexesMetadataResponse); // Deletes an index - rpc delete_index(DeleteIndexRequest) returns (EmptyResponse); - - // Gets all splits from index. - rpc list_all_splits(ListAllSplitsRequest) returns (ListSplitsResponse); + rpc DeleteIndex(DeleteIndexRequest) returns (EmptyResponse); // Gets splits from index. - rpc list_splits(ListSplitsRequest) returns (ListSplitsResponse); + rpc ListSplits(ListSplitsRequest) returns (ListSplitsResponse); // Stages several splits. - rpc stage_splits(StageSplitsRequest) returns (EmptyResponse); + rpc StageSplits(StageSplitsRequest) returns (EmptyResponse); // Publishes split. - rpc publish_splits(PublishSplitsRequest) returns (EmptyResponse); + rpc PublishSplits(PublishSplitsRequest) returns (EmptyResponse); // Marks splits for deletion. - rpc mark_splits_for_deletion(MarkSplitsForDeletionRequest) returns (EmptyResponse); + rpc MarkSplitsForDeletion(MarkSplitsForDeletionRequest) returns (EmptyResponse); // Deletes splits. - rpc delete_splits(DeleteSplitsRequest) returns (EmptyResponse); + rpc DeleteSplits(DeleteSplitsRequest) returns (EmptyResponse); // Adds source. - rpc add_source(AddSourceRequest) returns (EmptyResponse); + rpc AddSource(AddSourceRequest) returns (EmptyResponse); // Toggles source. - rpc toggle_source(ToggleSourceRequest) returns (EmptyResponse); + rpc ToggleSource(ToggleSourceRequest) returns (EmptyResponse); // Removes source. - rpc delete_source(DeleteSourceRequest) returns (EmptyResponse); + rpc DeleteSource(DeleteSourceRequest) returns (EmptyResponse); // Resets source checkpoint. - rpc reset_source_checkpoint(ResetSourceCheckpointRequest) returns (EmptyResponse); + rpc ResetSourceCheckpoint(ResetSourceCheckpointRequest) returns (EmptyResponse); // Gets last opstamp for a given `index_id`. - rpc last_delete_opstamp(LastDeleteOpstampRequest) returns (LastDeleteOpstampResponse); + rpc LastDeleteOpstamp(LastDeleteOpstampRequest) returns (LastDeleteOpstampResponse); // Creates a delete task. - rpc create_delete_task(DeleteQuery) returns (DeleteTask); + rpc CreateDeleteTask(DeleteQuery) returns (DeleteTask); // Updates splits `delete_opstamp`. - rpc update_splits_delete_opstamp(UpdateSplitsDeleteOpstampRequest) returns (UpdateSplitsDeleteOpstampResponse); + rpc UpdateSplitsDeleteOpstamp(UpdateSplitsDeleteOpstampRequest) returns (UpdateSplitsDeleteOpstampResponse); // Lists delete tasks with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. - rpc list_delete_tasks(ListDeleteTasksRequest) returns (ListDeleteTasksResponse); + rpc ListDeleteTasks(ListDeleteTasksRequest) returns (ListDeleteTasksResponse); /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. - rpc list_stale_splits(ListStaleSplitsRequest) returns (ListSplitsResponse); + rpc ListStaleSplits(ListStaleSplitsRequest) returns (ListSplitsResponse); /// /// Shard API @@ -129,12 +126,12 @@ message CreateIndexResponse { string index_uid = 1; } -message ListIndexesMetadatasRequest { +message ListIndexesMetadataRequest { string query_json = 1; } -message ListIndexesMetadatasResponse { - string indexes_metadatas_serialized_json = 1; +message ListIndexesMetadataResponse { + string indexes_metadata_serialized_json = 1; } message DeleteIndexRequest { @@ -142,17 +139,14 @@ message DeleteIndexRequest { } message IndexMetadataRequest { - string index_id = 1; + optional string index_id = 1; + optional string index_uid = 2; } message IndexMetadataResponse { string index_metadata_serialized_json = 1; } -message ListAllSplitsRequest { - string index_uid = 1; -} - message ListSplitsRequest { string query_json = 1; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index 242d3f34fe3..c4527d7f865 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -100,6 +100,12 @@ impl ControlPlaneServiceClient { where T: ControlPlaneService, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockControlPlaneService > (), + "`MockControlPlaneService` must be wrapped in a `MockControlPlaneServiceWrapper`. Use `MockControlPlaneService::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index 4758e8a895b..8ea5cb026e9 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -50,6 +50,12 @@ impl IndexingServiceClient { where T: IndexingService, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockIndexingService > (), + "`MockIndexingService` must be wrapped in a `MockIndexingServiceWrapper`. Use `MockIndexingService::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs index 88a8f15d503..3e079a9aa20 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -332,6 +332,12 @@ impl IngesterServiceClient { where T: IngesterService, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockIngesterService > (), + "`MockIngesterService` must be wrapped in a `MockIngesterServiceWrapper`. Use `MockIngesterService::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index 2c7b7331885..998532ad3e7 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -80,6 +80,12 @@ impl IngestRouterServiceClient { where T: IngestRouterService, { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockIngestRouterService > (), + "`MockIngestRouterService` must be wrapped in a `MockIngestRouterServiceWrapper`. Use `MockIngestRouterService::from(mock)` to instantiate the client." + ); Self { inner: Box::new(instance) } } pub fn as_grpc_service( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index 9b86903e36c..c5ab1253f21 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -1,4 +1,4 @@ -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DocBatchV2 { @@ -7,7 +7,7 @@ pub struct DocBatchV2 { #[prost(uint32, repeated, tag = "2")] pub doc_lengths: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct MRecordBatch { @@ -18,8 +18,8 @@ pub struct MRecordBatch { #[prost(uint32, repeated, tag = "2")] pub mrecord_lengths: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] #[derive(Eq)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct Shard { @@ -55,7 +55,7 @@ pub struct Shard { #[serde(default, skip_serializing_if = "Option::is_none")] pub publish_token: ::core::option::Option<::prost::alloc::string::String>, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] @@ -86,7 +86,7 @@ impl CommitTypeV2 { } } } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 92cac9be663..9e43549e210 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -1,78 +1,73 @@ -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct EmptyResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CreateIndexRequest { #[prost(string, tag = "2")] pub index_config_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CreateIndexResponse { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListIndexesMetadatasRequest { +pub struct ListIndexesMetadataRequest { #[prost(string, tag = "1")] pub query_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListIndexesMetadatasResponse { +pub struct ListIndexesMetadataResponse { #[prost(string, tag = "1")] - pub indexes_metadatas_serialized_json: ::prost::alloc::string::String, + pub indexes_metadata_serialized_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteIndexRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IndexMetadataRequest { - #[prost(string, tag = "1")] - pub index_id: ::prost::alloc::string::String, + #[prost(string, optional, tag = "1")] + pub index_id: ::core::option::Option<::prost::alloc::string::String>, + #[prost(string, optional, tag = "2")] + pub index_uid: ::core::option::Option<::prost::alloc::string::String>, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IndexMetadataResponse { #[prost(string, tag = "1")] pub index_metadata_serialized_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListAllSplitsRequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, -} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListSplitsRequest { #[prost(string, tag = "1")] pub query_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListSplitsResponse { #[prost(string, tag = "1")] pub splits_serialized_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct StageSplitsRequest { @@ -81,7 +76,7 @@ pub struct StageSplitsRequest { #[prost(string, tag = "2")] pub split_metadata_list_serialized_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PublishSplitsRequest { @@ -98,7 +93,7 @@ pub struct PublishSplitsRequest { #[prost(string, optional, tag = "5")] pub publish_token_opt: ::core::option::Option<::prost::alloc::string::String>, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct MarkSplitsForDeletionRequest { @@ -107,7 +102,7 @@ pub struct MarkSplitsForDeletionRequest { #[prost(string, repeated, tag = "3")] pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteSplitsRequest { @@ -116,7 +111,7 @@ pub struct DeleteSplitsRequest { #[prost(string, repeated, tag = "3")] pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct AddSourceRequest { @@ -125,7 +120,7 @@ pub struct AddSourceRequest { #[prost(string, tag = "2")] pub source_config_json: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ToggleSourceRequest { @@ -136,7 +131,7 @@ pub struct ToggleSourceRequest { #[prost(bool, tag = "3")] pub enable: bool, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteSourceRequest { @@ -145,7 +140,7 @@ pub struct DeleteSourceRequest { #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ResetSourceCheckpointRequest { @@ -154,7 +149,7 @@ pub struct ResetSourceCheckpointRequest { #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteTask { @@ -165,7 +160,7 @@ pub struct DeleteTask { #[prost(message, optional, tag = "3")] pub delete_query: ::core::option::Option, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteQuery { @@ -187,7 +182,7 @@ pub struct DeleteQuery { #[serde(alias = "query")] pub query_ast: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct UpdateSplitsDeleteOpstampRequest { @@ -198,25 +193,25 @@ pub struct UpdateSplitsDeleteOpstampRequest { #[prost(uint64, tag = "3")] pub delete_opstamp: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct UpdateSplitsDeleteOpstampResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct LastDeleteOpstampRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct LastDeleteOpstampResponse { #[prost(uint64, tag = "1")] pub last_delete_opstamp: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListStaleSplitsRequest { @@ -227,7 +222,7 @@ pub struct ListStaleSplitsRequest { #[prost(uint64, tag = "3")] pub num_splits: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListDeleteTasksRequest { @@ -236,21 +231,21 @@ pub struct ListDeleteTasksRequest { #[prost(uint64, tag = "2")] pub opstamp_start: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListDeleteTasksResponse { #[prost(message, repeated, tag = "1")] pub delete_tasks: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct OpenShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct OpenShardsSubrequest { @@ -265,14 +260,14 @@ pub struct OpenShardsSubrequest { #[prost(uint64, tag = "5")] pub next_shard_id: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct OpenShardsResponse { #[prost(message, repeated, tag = "1")] pub subresponses: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct OpenShardsSubresponse { @@ -285,14 +280,14 @@ pub struct OpenShardsSubresponse { #[prost(uint64, tag = "4")] pub next_shard_id: u64, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct AcquireShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct AcquireShardsSubrequest { @@ -305,14 +300,14 @@ pub struct AcquireShardsSubrequest { #[prost(string, tag = "4")] pub publish_token: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct AcquireShardsResponse { #[prost(message, repeated, tag = "1")] pub subresponses: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct AcquireShardsSubresponse { @@ -323,14 +318,14 @@ pub struct AcquireShardsSubresponse { #[prost(message, repeated, tag = "3")] pub acquired_shards: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CloseShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CloseShardsSubrequest { @@ -345,7 +340,7 @@ pub struct CloseShardsSubrequest { #[prost(uint64, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CloseShardsResponse { @@ -354,7 +349,7 @@ pub struct CloseShardsResponse { #[prost(message, repeated, tag = "2")] pub failures: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CloseShardsSuccess { @@ -371,7 +366,7 @@ pub struct CloseShardsSuccess { #[prost(string, tag = "6")] pub publish_position_inclusive: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct CloseShardsFailure { @@ -386,7 +381,7 @@ pub struct CloseShardsFailure { #[prost(string, tag = "5")] pub failure_message: ::prost::alloc::string::String, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteShardsRequest { @@ -395,7 +390,7 @@ pub struct DeleteShardsRequest { #[prost(bool, tag = "2")] pub force: bool, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteShardsSubrequest { @@ -406,18 +401,18 @@ pub struct DeleteShardsSubrequest { #[prost(uint64, repeated, tag = "3")] pub shard_ids: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteShardsResponse {} -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ListShardsSubrequest { @@ -428,119 +423,3311 @@ pub struct ListShardsSubrequest { #[prost(enumeration = "super::ingest::ShardState", optional, tag = "3")] pub shard_state: ::core::option::Option, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListShardsResponse { - #[prost(message, repeated, tag = "1")] - pub subresponses: ::prost::alloc::vec::Vec, +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListShardsResponse { + #[prost(message, repeated, tag = "1")] + pub subresponses: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ListShardsSubresponse { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "3")] + pub shards: ::prost::alloc::vec::Vec, + #[prost(uint64, tag = "4")] + pub next_shard_id: u64, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum SourceType { + Cli = 0, + File = 1, + GcpPubsub = 2, + IngestV1 = 3, + IngestV2 = 4, + Kafka = 5, + Kinesis = 6, + Nats = 7, + Pulsar = 8, + Vec = 9, + Void = 10, +} +impl SourceType { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + SourceType::Cli => "CLI", + SourceType::File => "FILE", + SourceType::GcpPubsub => "GCP_PUBSUB", + SourceType::IngestV1 => "INGEST_V1", + SourceType::IngestV2 => "INGEST_V2", + SourceType::Kafka => "KAFKA", + SourceType::Kinesis => "KINESIS", + SourceType::Nats => "NATS", + SourceType::Pulsar => "PULSAR", + SourceType::Vec => "VEC", + SourceType::Void => "VOID", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "CLI" => Some(Self::Cli), + "FILE" => Some(Self::File), + "GCP_PUBSUB" => Some(Self::GcpPubsub), + "INGEST_V1" => Some(Self::IngestV1), + "INGEST_V2" => Some(Self::IngestV2), + "KAFKA" => Some(Self::Kafka), + "KINESIS" => Some(Self::Kinesis), + "NATS" => Some(Self::Nats), + "PULSAR" => Some(Self::Pulsar), + "VEC" => Some(Self::Vec), + "VOID" => Some(Self::Void), + _ => None, + } + } +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum CloseShardsFailureKind { + InvalidArgument = 0, + NotFound = 1, +} +impl CloseShardsFailureKind { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + CloseShardsFailureKind::InvalidArgument => "INVALID_ARGUMENT", + CloseShardsFailureKind::NotFound => "NOT_FOUND", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "INVALID_ARGUMENT" => Some(Self::InvalidArgument), + "NOT_FOUND" => Some(Self::NotFound), + _ => None, + } + } +} +/// BEGIN quickwit-codegen +use tower::{Layer, Service, ServiceExt}; +use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; +impl PrometheusLabels<1> for CreateIndexRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("create_index")]) + } +} +impl PrometheusLabels<1> for IndexMetadataRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("index_metadata")]) + } +} +impl PrometheusLabels<1> for ListIndexesMetadataRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_indexes_metadata")]) + } +} +impl PrometheusLabels<1> for DeleteIndexRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_index")]) + } +} +impl PrometheusLabels<1> for ListSplitsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_splits")]) + } +} +impl PrometheusLabels<1> for StageSplitsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("stage_splits")]) + } +} +impl PrometheusLabels<1> for PublishSplitsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("publish_splits")]) + } +} +impl PrometheusLabels<1> for MarkSplitsForDeletionRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([ + std::borrow::Cow::Borrowed("mark_splits_for_deletion"), + ]) + } +} +impl PrometheusLabels<1> for DeleteSplitsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_splits")]) + } +} +impl PrometheusLabels<1> for AddSourceRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("add_source")]) + } +} +impl PrometheusLabels<1> for ToggleSourceRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("toggle_source")]) + } +} +impl PrometheusLabels<1> for DeleteSourceRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_source")]) + } +} +impl PrometheusLabels<1> for ResetSourceCheckpointRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([ + std::borrow::Cow::Borrowed("reset_source_checkpoint"), + ]) + } +} +impl PrometheusLabels<1> for LastDeleteOpstampRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("last_delete_opstamp")]) + } +} +impl PrometheusLabels<1> for DeleteQuery { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_query")]) + } +} +impl PrometheusLabels<1> for UpdateSplitsDeleteOpstampRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([ + std::borrow::Cow::Borrowed("update_splits_delete_opstamp"), + ]) + } +} +impl PrometheusLabels<1> for ListDeleteTasksRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_delete_tasks")]) + } +} +impl PrometheusLabels<1> for ListStaleSplitsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_stale_splits")]) + } +} +impl PrometheusLabels<1> for OpenShardsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("open_shards")]) + } +} +impl PrometheusLabels<1> for AcquireShardsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("acquire_shards")]) + } +} +impl PrometheusLabels<1> for CloseShardsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("close_shards")]) + } +} +impl PrometheusLabels<1> for DeleteShardsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_shards")]) + } +} +impl PrometheusLabels<1> for ListShardsRequest { + fn labels(&self) -> OwnedPrometheusLabels<1usize> { + OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("list_shards")]) + } +} +#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] +#[async_trait::async_trait] +pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { + /// Creates an index. + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> crate::metastore::MetastoreResult; + /// Gets an index metadata. + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult; + /// Gets an indexes metadatas. + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult; + /// Deletes an index + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult; + /// Gets splits from index. + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> crate::metastore::MetastoreResult; + /// Stages several splits. + async fn stage_splits( + &mut self, + request: StageSplitsRequest, + ) -> crate::metastore::MetastoreResult; + /// Publishes split. + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult; + /// Marks splits for deletion. + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult; + /// Deletes splits. + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult; + /// Adds source. + async fn add_source( + &mut self, + request: AddSourceRequest, + ) -> crate::metastore::MetastoreResult; + /// Toggles source. + async fn toggle_source( + &mut self, + request: ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult; + /// Removes source. + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult; + /// Resets source checkpoint. + async fn reset_source_checkpoint( + &mut self, + request: ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult; + /// Gets last opstamp for a given `index_id`. + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult; + /// Creates a delete task. + async fn create_delete_task( + &mut self, + request: DeleteQuery, + ) -> crate::metastore::MetastoreResult; + /// Updates splits `delete_opstamp`. + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult; + /// Lists delete tasks with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. + async fn list_delete_tasks( + &mut self, + request: ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult; + #[doc = "/ Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`."] + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult; + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> crate::metastore::MetastoreResult; + /// Acquires a set of shards for indexing. This RPC locks the shards for publishing thanks to a publish token and only + /// the last indexer that has acquired the shards is allowed to publish. The response returns for each subrequest the + /// list of acquired shards along with the positions to index from. + async fn acquire_shards( + &mut self, + request: AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult; + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::metastore::MetastoreResult; + async fn delete_shards( + &mut self, + request: DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult; + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> crate::metastore::MetastoreResult; + async fn check_connectivity(&mut self) -> anyhow::Result<()>; + fn endpoints(&self) -> Vec; +} +dyn_clone::clone_trait_object!(MetastoreService); +#[cfg(any(test, feature = "testsuite"))] +impl Clone for MockMetastoreService { + fn clone(&self) -> Self { + MockMetastoreService::new() + } +} +#[derive(Debug, Clone)] +pub struct MetastoreServiceClient { + inner: Box, +} +impl MetastoreServiceClient { + pub fn new(instance: T) -> Self + where + T: MetastoreService, + { + #[cfg(any(test, feature = "testsuite"))] + assert!( + std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < + MockMetastoreService > (), + "`MockMetastoreService` must be wrapped in a `MockMetastoreServiceWrapper`. Use `MockMetastoreService::from(mock)` to instantiate the client." + ); + Self { inner: Box::new(instance) } + } + pub fn as_grpc_service( + &self, + ) -> metastore_service_grpc_server::MetastoreServiceGrpcServer< + MetastoreServiceGrpcServerAdapter, + > { + let adapter = MetastoreServiceGrpcServerAdapter::new(self.clone()); + metastore_service_grpc_server::MetastoreServiceGrpcServer::new(adapter) + } + pub fn from_channel( + addr: std::net::SocketAddr, + channel: tonic::transport::Channel, + ) -> Self { + let (_, connection_keys_watcher) = tokio::sync::watch::channel( + std::collections::HashSet::from_iter([addr]), + ); + let adapter = MetastoreServiceGrpcClientAdapter::new( + metastore_service_grpc_client::MetastoreServiceGrpcClient::new(channel), + connection_keys_watcher, + ); + Self::new(adapter) + } + pub fn from_balance_channel( + balance_channel: quickwit_common::tower::BalanceChannel, + ) -> MetastoreServiceClient { + let connection_keys_watcher = balance_channel.connection_keys_watcher(); + let adapter = MetastoreServiceGrpcClientAdapter::new( + metastore_service_grpc_client::MetastoreServiceGrpcClient::new( + balance_channel, + ), + connection_keys_watcher, + ); + Self::new(adapter) + } + pub fn from_mailbox(mailbox: quickwit_actors::Mailbox) -> Self + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + MetastoreServiceMailbox: MetastoreService, + { + MetastoreServiceClient::new(MetastoreServiceMailbox::new(mailbox)) + } + pub fn tower() -> MetastoreServiceTowerBlockBuilder { + MetastoreServiceTowerBlockBuilder::default() + } + #[cfg(any(test, feature = "testsuite"))] + pub fn mock() -> MockMetastoreService { + MockMetastoreService::new() + } +} +#[async_trait::async_trait] +impl MetastoreService for MetastoreServiceClient { + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.create_index(request).await + } + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.index_metadata(request).await + } + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.list_indexes_metadata(request).await + } + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.delete_index(request).await + } + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.list_splits(request).await + } + async fn stage_splits( + &mut self, + request: StageSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.stage_splits(request).await + } + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.publish_splits(request).await + } + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.mark_splits_for_deletion(request).await + } + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.delete_splits(request).await + } + async fn add_source( + &mut self, + request: AddSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.add_source(request).await + } + async fn toggle_source( + &mut self, + request: ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.toggle_source(request).await + } + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.delete_source(request).await + } + async fn reset_source_checkpoint( + &mut self, + request: ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.reset_source_checkpoint(request).await + } + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.last_delete_opstamp(request).await + } + async fn create_delete_task( + &mut self, + request: DeleteQuery, + ) -> crate::metastore::MetastoreResult { + self.inner.create_delete_task(request).await + } + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.update_splits_delete_opstamp(request).await + } + async fn list_delete_tasks( + &mut self, + request: ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.list_delete_tasks(request).await + } + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.list_stale_splits(request).await + } + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.open_shards(request).await + } + async fn acquire_shards( + &mut self, + request: AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.acquire_shards(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.close_shards(request).await + } + async fn delete_shards( + &mut self, + request: DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.delete_shards(request).await + } + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.list_shards(request).await + } + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + self.inner.check_connectivity().await + } + fn endpoints(&self) -> Vec { + self.inner.endpoints() + } +} +#[cfg(any(test, feature = "testsuite"))] +pub mod metastore_service_mock { + use super::*; + #[derive(Debug, Clone)] + struct MockMetastoreServiceWrapper { + inner: std::sync::Arc>, + } + #[async_trait::async_trait] + impl MetastoreService for MockMetastoreServiceWrapper { + async fn create_index( + &mut self, + request: super::CreateIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.create_index(request).await + } + async fn index_metadata( + &mut self, + request: super::IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.index_metadata(request).await + } + async fn list_indexes_metadata( + &mut self, + request: super::ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_indexes_metadata(request).await + } + async fn delete_index( + &mut self, + request: super::DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_index(request).await + } + async fn list_splits( + &mut self, + request: super::ListSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_splits(request).await + } + async fn stage_splits( + &mut self, + request: super::StageSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.stage_splits(request).await + } + async fn publish_splits( + &mut self, + request: super::PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.publish_splits(request).await + } + async fn mark_splits_for_deletion( + &mut self, + request: super::MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.mark_splits_for_deletion(request).await + } + async fn delete_splits( + &mut self, + request: super::DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_splits(request).await + } + async fn add_source( + &mut self, + request: super::AddSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.add_source(request).await + } + async fn toggle_source( + &mut self, + request: super::ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.toggle_source(request).await + } + async fn delete_source( + &mut self, + request: super::DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_source(request).await + } + async fn reset_source_checkpoint( + &mut self, + request: super::ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.reset_source_checkpoint(request).await + } + async fn last_delete_opstamp( + &mut self, + request: super::LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.last_delete_opstamp(request).await + } + async fn create_delete_task( + &mut self, + request: super::DeleteQuery, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.create_delete_task(request).await + } + async fn update_splits_delete_opstamp( + &mut self, + request: super::UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult< + super::UpdateSplitsDeleteOpstampResponse, + > { + self.inner.lock().await.update_splits_delete_opstamp(request).await + } + async fn list_delete_tasks( + &mut self, + request: super::ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_delete_tasks(request).await + } + async fn list_stale_splits( + &mut self, + request: super::ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_stale_splits(request).await + } + async fn open_shards( + &mut self, + request: super::OpenShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.open_shards(request).await + } + async fn acquire_shards( + &mut self, + request: super::AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.acquire_shards(request).await + } + async fn close_shards( + &mut self, + request: super::CloseShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.close_shards(request).await + } + async fn delete_shards( + &mut self, + request: super::DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.delete_shards(request).await + } + async fn list_shards( + &mut self, + request: super::ListShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.list_shards(request).await + } + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + self.inner.lock().await.check_connectivity().await + } + fn endpoints(&self) -> Vec { + futures::executor::block_on(self.inner.lock()).endpoints() + } + } + impl From for MetastoreServiceClient { + fn from(mock: MockMetastoreService) -> Self { + let mock_wrapper = MockMetastoreServiceWrapper { + inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + }; + MetastoreServiceClient::new(mock_wrapper) + } + } +} +pub type BoxFuture = std::pin::Pin< + Box> + Send + 'static>, +>; +impl tower::Service for Box { + type Response = CreateIndexResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: CreateIndexRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.create_index(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = IndexMetadataResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: IndexMetadataRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.index_metadata(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = ListIndexesMetadataResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ListIndexesMetadataRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.list_indexes_metadata(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteIndexRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.delete_index(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = ListSplitsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ListSplitsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.list_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: StageSplitsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.stage_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: PublishSplitsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.publish_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: MarkSplitsForDeletionRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.mark_splits_for_deletion(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteSplitsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.delete_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: AddSourceRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.add_source(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ToggleSourceRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.toggle_source(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteSourceRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.delete_source(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ResetSourceCheckpointRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.reset_source_checkpoint(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = LastDeleteOpstampResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: LastDeleteOpstampRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.last_delete_opstamp(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = DeleteTask; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteQuery) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.create_delete_task(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = UpdateSplitsDeleteOpstampResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: UpdateSplitsDeleteOpstampRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.update_splits_delete_opstamp(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = ListDeleteTasksResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ListDeleteTasksRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.list_delete_tasks(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = ListSplitsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ListStaleSplitsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.list_stale_splits(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = OpenShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: OpenShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.open_shards(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = AcquireShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: AcquireShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.acquire_shards(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = CloseShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: CloseShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.close_shards(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = DeleteShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: DeleteShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.delete_shards(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = ListShardsResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: ListShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.list_shards(request).await }; + Box::pin(fut) + } +} +/// A tower block is a set of towers. Each tower is stack of layers (middlewares) that are applied to a service. +#[derive(Debug)] +struct MetastoreServiceTowerBlock { + inner: Box, + create_index_svc: quickwit_common::tower::BoxService< + CreateIndexRequest, + CreateIndexResponse, + crate::metastore::MetastoreError, + >, + index_metadata_svc: quickwit_common::tower::BoxService< + IndexMetadataRequest, + IndexMetadataResponse, + crate::metastore::MetastoreError, + >, + list_indexes_metadata_svc: quickwit_common::tower::BoxService< + ListIndexesMetadataRequest, + ListIndexesMetadataResponse, + crate::metastore::MetastoreError, + >, + delete_index_svc: quickwit_common::tower::BoxService< + DeleteIndexRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + list_splits_svc: quickwit_common::tower::BoxService< + ListSplitsRequest, + ListSplitsResponse, + crate::metastore::MetastoreError, + >, + stage_splits_svc: quickwit_common::tower::BoxService< + StageSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + publish_splits_svc: quickwit_common::tower::BoxService< + PublishSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + mark_splits_for_deletion_svc: quickwit_common::tower::BoxService< + MarkSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + delete_splits_svc: quickwit_common::tower::BoxService< + DeleteSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + add_source_svc: quickwit_common::tower::BoxService< + AddSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + toggle_source_svc: quickwit_common::tower::BoxService< + ToggleSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + delete_source_svc: quickwit_common::tower::BoxService< + DeleteSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + reset_source_checkpoint_svc: quickwit_common::tower::BoxService< + ResetSourceCheckpointRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + last_delete_opstamp_svc: quickwit_common::tower::BoxService< + LastDeleteOpstampRequest, + LastDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + create_delete_task_svc: quickwit_common::tower::BoxService< + DeleteQuery, + DeleteTask, + crate::metastore::MetastoreError, + >, + update_splits_delete_opstamp_svc: quickwit_common::tower::BoxService< + UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + list_delete_tasks_svc: quickwit_common::tower::BoxService< + ListDeleteTasksRequest, + ListDeleteTasksResponse, + crate::metastore::MetastoreError, + >, + list_stale_splits_svc: quickwit_common::tower::BoxService< + ListStaleSplitsRequest, + ListSplitsResponse, + crate::metastore::MetastoreError, + >, + open_shards_svc: quickwit_common::tower::BoxService< + OpenShardsRequest, + OpenShardsResponse, + crate::metastore::MetastoreError, + >, + acquire_shards_svc: quickwit_common::tower::BoxService< + AcquireShardsRequest, + AcquireShardsResponse, + crate::metastore::MetastoreError, + >, + close_shards_svc: quickwit_common::tower::BoxService< + CloseShardsRequest, + CloseShardsResponse, + crate::metastore::MetastoreError, + >, + delete_shards_svc: quickwit_common::tower::BoxService< + DeleteShardsRequest, + DeleteShardsResponse, + crate::metastore::MetastoreError, + >, + list_shards_svc: quickwit_common::tower::BoxService< + ListShardsRequest, + ListShardsResponse, + crate::metastore::MetastoreError, + >, +} +impl Clone for MetastoreServiceTowerBlock { + fn clone(&self) -> Self { + Self { + inner: self.inner.clone(), + create_index_svc: self.create_index_svc.clone(), + index_metadata_svc: self.index_metadata_svc.clone(), + list_indexes_metadata_svc: self.list_indexes_metadata_svc.clone(), + delete_index_svc: self.delete_index_svc.clone(), + list_splits_svc: self.list_splits_svc.clone(), + stage_splits_svc: self.stage_splits_svc.clone(), + publish_splits_svc: self.publish_splits_svc.clone(), + mark_splits_for_deletion_svc: self.mark_splits_for_deletion_svc.clone(), + delete_splits_svc: self.delete_splits_svc.clone(), + add_source_svc: self.add_source_svc.clone(), + toggle_source_svc: self.toggle_source_svc.clone(), + delete_source_svc: self.delete_source_svc.clone(), + reset_source_checkpoint_svc: self.reset_source_checkpoint_svc.clone(), + last_delete_opstamp_svc: self.last_delete_opstamp_svc.clone(), + create_delete_task_svc: self.create_delete_task_svc.clone(), + update_splits_delete_opstamp_svc: self + .update_splits_delete_opstamp_svc + .clone(), + list_delete_tasks_svc: self.list_delete_tasks_svc.clone(), + list_stale_splits_svc: self.list_stale_splits_svc.clone(), + open_shards_svc: self.open_shards_svc.clone(), + acquire_shards_svc: self.acquire_shards_svc.clone(), + close_shards_svc: self.close_shards_svc.clone(), + delete_shards_svc: self.delete_shards_svc.clone(), + list_shards_svc: self.list_shards_svc.clone(), + } + } +} +#[async_trait::async_trait] +impl MetastoreService for MetastoreServiceTowerBlock { + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.create_index_svc.ready().await?.call(request).await + } + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.index_metadata_svc.ready().await?.call(request).await + } + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.list_indexes_metadata_svc.ready().await?.call(request).await + } + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_index_svc.ready().await?.call(request).await + } + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.list_splits_svc.ready().await?.call(request).await + } + async fn stage_splits( + &mut self, + request: StageSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.stage_splits_svc.ready().await?.call(request).await + } + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.publish_splits_svc.ready().await?.call(request).await + } + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.mark_splits_for_deletion_svc.ready().await?.call(request).await + } + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_splits_svc.ready().await?.call(request).await + } + async fn add_source( + &mut self, + request: AddSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.add_source_svc.ready().await?.call(request).await + } + async fn toggle_source( + &mut self, + request: ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.toggle_source_svc.ready().await?.call(request).await + } + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_source_svc.ready().await?.call(request).await + } + async fn reset_source_checkpoint( + &mut self, + request: ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult { + self.reset_source_checkpoint_svc.ready().await?.call(request).await + } + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.last_delete_opstamp_svc.ready().await?.call(request).await + } + async fn create_delete_task( + &mut self, + request: DeleteQuery, + ) -> crate::metastore::MetastoreResult { + self.create_delete_task_svc.ready().await?.call(request).await + } + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.update_splits_delete_opstamp_svc.ready().await?.call(request).await + } + async fn list_delete_tasks( + &mut self, + request: ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult { + self.list_delete_tasks_svc.ready().await?.call(request).await + } + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.list_stale_splits_svc.ready().await?.call(request).await + } + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.open_shards_svc.ready().await?.call(request).await + } + async fn acquire_shards( + &mut self, + request: AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.acquire_shards_svc.ready().await?.call(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.close_shards_svc.ready().await?.call(request).await + } + async fn delete_shards( + &mut self, + request: DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.delete_shards_svc.ready().await?.call(request).await + } + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.list_shards_svc.ready().await?.call(request).await + } + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + self.inner.check_connectivity().await + } + fn endpoints(&self) -> Vec { + self.inner.endpoints() + } +} +#[derive(Debug, Default)] +pub struct MetastoreServiceTowerBlockBuilder { + #[allow(clippy::type_complexity)] + create_index_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + CreateIndexRequest, + CreateIndexResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + index_metadata_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + IndexMetadataRequest, + IndexMetadataResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + list_indexes_metadata_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ListIndexesMetadataRequest, + ListIndexesMetadataResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + delete_index_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + DeleteIndexRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + list_splits_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ListSplitsRequest, + ListSplitsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + stage_splits_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + StageSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + publish_splits_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + PublishSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + mark_splits_for_deletion_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + MarkSplitsForDeletionRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + delete_splits_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + DeleteSplitsRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + add_source_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + AddSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + toggle_source_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ToggleSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + delete_source_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + DeleteSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + reset_source_checkpoint_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ResetSourceCheckpointRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + last_delete_opstamp_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + LastDeleteOpstampRequest, + LastDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + create_delete_task_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + DeleteQuery, + DeleteTask, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + update_splits_delete_opstamp_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + list_delete_tasks_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ListDeleteTasksRequest, + ListDeleteTasksResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + list_stale_splits_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ListStaleSplitsRequest, + ListSplitsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + open_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + OpenShardsRequest, + OpenShardsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + acquire_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + AcquireShardsRequest, + AcquireShardsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + close_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + CloseShardsRequest, + CloseShardsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + delete_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + DeleteShardsRequest, + DeleteShardsResponse, + crate::metastore::MetastoreError, + >, + >, + #[allow(clippy::type_complexity)] + list_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + ListShardsRequest, + ListShardsResponse, + crate::metastore::MetastoreError, + >, + >, +} +impl MetastoreServiceTowerBlockBuilder { + pub fn shared_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Clone + Send + Sync + 'static, + L::Service: tower::Service< + CreateIndexRequest, + Response = CreateIndexResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + IndexMetadataRequest, + Response = IndexMetadataResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ListIndexesMetadataRequest, + Response = ListIndexesMetadataResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + DeleteIndexRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ListSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + StageSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + PublishSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + MarkSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + DeleteSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + AddSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ToggleSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + DeleteSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ResetSourceCheckpointRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + LastDeleteOpstampRequest, + Response = LastDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + DeleteQuery, + Response = DeleteTask, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + UpdateSplitsDeleteOpstampRequest, + Response = UpdateSplitsDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ListDeleteTasksRequest, + Response = ListDeleteTasksResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ListStaleSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + OpenShardsRequest, + Response = OpenShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + AcquireShardsRequest, + Response = AcquireShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + CloseShardsRequest, + Response = CloseShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + DeleteShardsRequest, + Response = DeleteShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + ListShardsRequest, + Response = ListShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .create_index_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .index_metadata_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .list_indexes_metadata_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .delete_index_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .list_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .stage_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .publish_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .mark_splits_for_deletion_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .delete_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .add_source_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .toggle_source_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .delete_source_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .reset_source_checkpoint_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .last_delete_opstamp_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .create_delete_task_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .update_splits_delete_opstamp_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .list_delete_tasks_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .list_stale_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .open_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .acquire_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .close_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .delete_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self.list_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn create_index_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + CreateIndexRequest, + Response = CreateIndexResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.create_index_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn index_metadata_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + IndexMetadataRequest, + Response = IndexMetadataResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.index_metadata_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn list_indexes_metadata_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ListIndexesMetadataRequest, + Response = ListIndexesMetadataResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .list_indexes_metadata_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn delete_index_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + DeleteIndexRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_index_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn list_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ListSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.list_splits_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn stage_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + StageSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.stage_splits_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn publish_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + PublishSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.publish_splits_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn mark_splits_for_deletion_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + MarkSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .mark_splits_for_deletion_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn delete_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + DeleteSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_splits_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn add_source_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + AddSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.add_source_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn toggle_source_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ToggleSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.toggle_source_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn delete_source_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + DeleteSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_source_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn reset_source_checkpoint_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ResetSourceCheckpointRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .reset_source_checkpoint_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn last_delete_opstamp_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + LastDeleteOpstampRequest, + Response = LastDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .last_delete_opstamp_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn create_delete_task_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + DeleteQuery, + Response = DeleteTask, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .create_delete_task_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn update_splits_delete_opstamp_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + UpdateSplitsDeleteOpstampRequest, + Response = UpdateSplitsDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .update_splits_delete_opstamp_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn list_delete_tasks_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ListDeleteTasksRequest, + Response = ListDeleteTasksResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .list_delete_tasks_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn list_stale_splits_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ListStaleSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .list_stale_splits_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn open_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + OpenShardsRequest, + Response = OpenShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.open_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn acquire_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + AcquireShardsRequest, + Response = AcquireShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.acquire_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn close_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + CloseShardsRequest, + Response = CloseShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn delete_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + DeleteShardsRequest, + Response = DeleteShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn list_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + ListShardsRequest, + Response = ListShardsResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.list_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn build(self, instance: T) -> MetastoreServiceClient + where + T: MetastoreService, + { + self.build_from_boxed(Box::new(instance)) + } + pub fn build_from_channel( + self, + addr: std::net::SocketAddr, + channel: tonic::transport::Channel, + ) -> MetastoreServiceClient { + self.build_from_boxed( + Box::new(MetastoreServiceClient::from_channel(addr, channel)), + ) + } + pub fn build_from_balance_channel( + self, + balance_channel: quickwit_common::tower::BalanceChannel, + ) -> MetastoreServiceClient { + self.build_from_boxed( + Box::new(MetastoreServiceClient::from_balance_channel(balance_channel)), + ) + } + pub fn build_from_mailbox( + self, + mailbox: quickwit_actors::Mailbox, + ) -> MetastoreServiceClient + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + MetastoreServiceMailbox: MetastoreService, + { + self.build_from_boxed(Box::new(MetastoreServiceMailbox::new(mailbox))) + } + fn build_from_boxed( + self, + boxed_instance: Box, + ) -> MetastoreServiceClient { + let create_index_svc = if let Some(layer) = self.create_index_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let index_metadata_svc = if let Some(layer) = self.index_metadata_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let list_indexes_metadata_svc = if let Some(layer) + = self.list_indexes_metadata_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let delete_index_svc = if let Some(layer) = self.delete_index_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let list_splits_svc = if let Some(layer) = self.list_splits_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let stage_splits_svc = if let Some(layer) = self.stage_splits_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let publish_splits_svc = if let Some(layer) = self.publish_splits_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let mark_splits_for_deletion_svc = if let Some(layer) + = self.mark_splits_for_deletion_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let delete_splits_svc = if let Some(layer) = self.delete_splits_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let add_source_svc = if let Some(layer) = self.add_source_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let toggle_source_svc = if let Some(layer) = self.toggle_source_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let delete_source_svc = if let Some(layer) = self.delete_source_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let reset_source_checkpoint_svc = if let Some(layer) + = self.reset_source_checkpoint_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let last_delete_opstamp_svc = if let Some(layer) = self.last_delete_opstamp_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let create_delete_task_svc = if let Some(layer) = self.create_delete_task_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let update_splits_delete_opstamp_svc = if let Some(layer) + = self.update_splits_delete_opstamp_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let list_delete_tasks_svc = if let Some(layer) = self.list_delete_tasks_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let list_stale_splits_svc = if let Some(layer) = self.list_stale_splits_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let open_shards_svc = if let Some(layer) = self.open_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let acquire_shards_svc = if let Some(layer) = self.acquire_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let close_shards_svc = if let Some(layer) = self.close_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let delete_shards_svc = if let Some(layer) = self.delete_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let list_shards_svc = if let Some(layer) = self.list_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let tower_block = MetastoreServiceTowerBlock { + inner: boxed_instance.clone(), + create_index_svc, + index_metadata_svc, + list_indexes_metadata_svc, + delete_index_svc, + list_splits_svc, + stage_splits_svc, + publish_splits_svc, + mark_splits_for_deletion_svc, + delete_splits_svc, + add_source_svc, + toggle_source_svc, + delete_source_svc, + reset_source_checkpoint_svc, + last_delete_opstamp_svc, + create_delete_task_svc, + update_splits_delete_opstamp_svc, + list_delete_tasks_svc, + list_stale_splits_svc, + open_shards_svc, + acquire_shards_svc, + close_shards_svc, + delete_shards_svc, + list_shards_svc, + }; + MetastoreServiceClient::new(tower_block) + } +} +#[derive(Debug, Clone)] +struct MailboxAdapter { + inner: quickwit_actors::Mailbox, + phantom: std::marker::PhantomData, } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct ListShardsSubresponse { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - #[prost(message, repeated, tag = "3")] - pub shards: ::prost::alloc::vec::Vec, - #[prost(uint64, tag = "4")] - pub next_shard_id: u64, +impl std::ops::Deref for MailboxAdapter +where + A: quickwit_actors::Actor, +{ + type Target = quickwit_actors::Mailbox; + fn deref(&self) -> &Self::Target { + &self.inner + } } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[serde(rename_all = "snake_case")] -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum SourceType { - Cli = 0, - File = 1, - GcpPubsub = 2, - IngestV1 = 3, - IngestV2 = 4, - Kafka = 5, - Kinesis = 6, - Nats = 7, - Pulsar = 8, - Vec = 9, - Void = 10, +#[derive(Debug)] +pub struct MetastoreServiceMailbox { + inner: MailboxAdapter, } -impl SourceType { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - SourceType::Cli => "CLI", - SourceType::File => "FILE", - SourceType::GcpPubsub => "GCP_PUBSUB", - SourceType::IngestV1 => "INGEST_V1", - SourceType::IngestV2 => "INGEST_V2", - SourceType::Kafka => "KAFKA", - SourceType::Kinesis => "KINESIS", - SourceType::Nats => "NATS", - SourceType::Pulsar => "PULSAR", - SourceType::Vec => "VEC", - SourceType::Void => "VOID", - } +impl MetastoreServiceMailbox { + pub fn new(instance: quickwit_actors::Mailbox) -> Self { + let inner = MailboxAdapter { + inner: instance, + phantom: std::marker::PhantomData, + }; + Self { inner } } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "CLI" => Some(Self::Cli), - "FILE" => Some(Self::File), - "GCP_PUBSUB" => Some(Self::GcpPubsub), - "INGEST_V1" => Some(Self::IngestV1), - "INGEST_V2" => Some(Self::IngestV2), - "KAFKA" => Some(Self::Kafka), - "KINESIS" => Some(Self::Kinesis), - "NATS" => Some(Self::Nats), - "PULSAR" => Some(Self::Pulsar), - "VEC" => Some(Self::Vec), - "VOID" => Some(Self::Void), - _ => None, +} +impl Clone for MetastoreServiceMailbox { + fn clone(&self) -> Self { + let inner = MailboxAdapter { + inner: self.inner.clone(), + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl tower::Service for MetastoreServiceMailbox +where + A: quickwit_actors::Actor + + quickwit_actors::DeferableReplyHandler> + Send + + 'static, + M: std::fmt::Debug + Send + 'static, + T: Send + 'static, + E: std::fmt::Debug + Send + 'static, + crate::metastore::MetastoreError: From>, +{ + type Response = T; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + //! This does not work with balance middlewares such as `tower::balance::pool::Pool` because + //! this always returns `Poll::Ready`. The fix is to acquire a permit from the + //! mailbox in `poll_ready` and consume it in `call`. + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, message: M) -> Self::Future { + let mailbox = self.inner.clone(); + let fut = async move { + mailbox.ask_for_res(message).await.map_err(|error| error.into()) + }; + Box::pin(fut) + } +} +#[async_trait::async_trait] +impl MetastoreService for MetastoreServiceMailbox +where + A: quickwit_actors::Actor + std::fmt::Debug, + MetastoreServiceMailbox< + A, + >: tower::Service< + CreateIndexRequest, + Response = CreateIndexResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + IndexMetadataRequest, + Response = IndexMetadataResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ListIndexesMetadataRequest, + Response = ListIndexesMetadataResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture< + ListIndexesMetadataResponse, + crate::metastore::MetastoreError, + >, + > + + tower::Service< + DeleteIndexRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ListSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + StageSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + PublishSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + MarkSplitsForDeletionRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + DeleteSplitsRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + AddSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ToggleSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + DeleteSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ResetSourceCheckpointRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + LastDeleteOpstampRequest, + Response = LastDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture< + LastDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + > + + tower::Service< + DeleteQuery, + Response = DeleteTask, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + UpdateSplitsDeleteOpstampRequest, + Response = UpdateSplitsDeleteOpstampResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture< + UpdateSplitsDeleteOpstampResponse, + crate::metastore::MetastoreError, + >, + > + + tower::Service< + ListDeleteTasksRequest, + Response = ListDeleteTasksResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ListStaleSplitsRequest, + Response = ListSplitsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + OpenShardsRequest, + Response = OpenShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + AcquireShardsRequest, + Response = AcquireShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + CloseShardsRequest, + Response = CloseShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + DeleteShardsRequest, + Response = DeleteShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + + tower::Service< + ListShardsRequest, + Response = ListShardsResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + >, +{ + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn stage_splits( + &mut self, + request: StageSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn add_source( + &mut self, + request: AddSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn toggle_source( + &mut self, + request: ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn reset_source_checkpoint( + &mut self, + request: ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn create_delete_task( + &mut self, + request: DeleteQuery, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn list_delete_tasks( + &mut self, + request: ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn acquire_shards( + &mut self, + request: AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn delete_shards( + &mut self, + request: DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.call(request).await + } + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + if self.inner.is_disconnected() { + anyhow::bail!("actor `{}` is disconnected", self.inner.actor_instance_id()) } + Ok(()) + } + fn endpoints(&self) -> Vec { + vec![ + quickwit_common::uri::Uri::from_well_formed(format!("actor://localhost/{}", + self.inner.actor_instance_id())) + ] } } -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[serde(rename_all = "snake_case")] -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum CloseShardsFailureKind { - InvalidArgument = 0, - NotFound = 1, +#[derive(Debug, Clone)] +pub struct MetastoreServiceGrpcClientAdapter { + inner: T, + #[allow(dead_code)] + connection_addrs_rx: tokio::sync::watch::Receiver< + std::collections::HashSet, + >, } -impl CloseShardsFailureKind { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - CloseShardsFailureKind::InvalidArgument => "INVALID_ARGUMENT", - CloseShardsFailureKind::NotFound => "NOT_FOUND", +impl MetastoreServiceGrpcClientAdapter { + pub fn new( + instance: T, + connection_addrs_rx: tokio::sync::watch::Receiver< + std::collections::HashSet, + >, + ) -> Self { + Self { + inner: instance, + connection_addrs_rx, } } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "INVALID_ARGUMENT" => Some(Self::InvalidArgument), - "NOT_FOUND" => Some(Self::NotFound), - _ => None, +} +#[async_trait::async_trait] +impl MetastoreService +for MetastoreServiceGrpcClientAdapter< + metastore_service_grpc_client::MetastoreServiceGrpcClient, +> +where + T: tonic::client::GrpcService + std::fmt::Debug + Clone + Send + + Sync + 'static, + T::ResponseBody: tonic::codegen::Body + Send + 'static, + ::Error: Into + + Send, + T::Future: Send, +{ + async fn create_index( + &mut self, + request: CreateIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .create_index(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn index_metadata( + &mut self, + request: IndexMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .index_metadata(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn list_indexes_metadata( + &mut self, + request: ListIndexesMetadataRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .list_indexes_metadata(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn delete_index( + &mut self, + request: DeleteIndexRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .delete_index(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn list_splits( + &mut self, + request: ListSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .list_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn stage_splits( + &mut self, + request: StageSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .stage_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn publish_splits( + &mut self, + request: PublishSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .publish_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn mark_splits_for_deletion( + &mut self, + request: MarkSplitsForDeletionRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .mark_splits_for_deletion(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn delete_splits( + &mut self, + request: DeleteSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .delete_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn add_source( + &mut self, + request: AddSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .add_source(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn toggle_source( + &mut self, + request: ToggleSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .toggle_source(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn delete_source( + &mut self, + request: DeleteSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .delete_source(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn reset_source_checkpoint( + &mut self, + request: ResetSourceCheckpointRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .reset_source_checkpoint(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn last_delete_opstamp( + &mut self, + request: LastDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .last_delete_opstamp(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn create_delete_task( + &mut self, + request: DeleteQuery, + ) -> crate::metastore::MetastoreResult { + self.inner + .create_delete_task(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn update_splits_delete_opstamp( + &mut self, + request: UpdateSplitsDeleteOpstampRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .update_splits_delete_opstamp(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn list_delete_tasks( + &mut self, + request: ListDeleteTasksRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .list_delete_tasks(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn list_stale_splits( + &mut self, + request: ListStaleSplitsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .list_stale_splits(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn open_shards( + &mut self, + request: OpenShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .open_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn acquire_shards( + &mut self, + request: AcquireShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .acquire_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .close_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn delete_shards( + &mut self, + request: DeleteShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .delete_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn list_shards( + &mut self, + request: ListShardsRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .list_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn check_connectivity(&mut self) -> anyhow::Result<()> { + if self.connection_addrs_rx.borrow().len() == 0 { + anyhow::bail!("no server currently available") } + Ok(()) + } + fn endpoints(&self) -> Vec { + self.connection_addrs_rx + .borrow() + .iter() + .map(|addr| quickwit_common::uri::Uri::from_well_formed( + format!( + r"grpc://{}/{}.{}", addr, "quickwit.metastore", "MetastoreService" + ), + )) + .collect() + } +} +#[derive(Debug)] +pub struct MetastoreServiceGrpcServerAdapter { + inner: Box, +} +impl MetastoreServiceGrpcServerAdapter { + pub fn new(instance: T) -> Self + where + T: MetastoreService, + { + Self { inner: Box::new(instance) } + } +} +#[async_trait::async_trait] +impl metastore_service_grpc_server::MetastoreServiceGrpc +for MetastoreServiceGrpcServerAdapter { + async fn create_index( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .create_index(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn index_metadata( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .index_metadata(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn list_indexes_metadata( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .list_indexes_metadata(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn delete_index( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .delete_index(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn list_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .list_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn stage_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .stage_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn publish_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .publish_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn mark_splits_for_deletion( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .mark_splits_for_deletion(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn delete_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .delete_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn add_source( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .add_source(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn toggle_source( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .toggle_source(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn delete_source( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .delete_source(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn reset_source_checkpoint( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .reset_source_checkpoint(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn last_delete_opstamp( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .last_delete_opstamp(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn create_delete_task( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .create_delete_task(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn update_splits_delete_opstamp( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .update_splits_delete_opstamp(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn list_delete_tasks( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .list_delete_tasks(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn list_stale_splits( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .list_stale_splits(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn open_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .open_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn acquire_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .acquire_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn close_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .close_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn delete_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .delete_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn list_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .list_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) } } /// Generated client implementations. -pub mod metastore_service_client { +pub mod metastore_service_grpc_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; use tonic::codegen::http::Uri; #[derive(Debug, Clone)] - pub struct MetastoreServiceClient { + pub struct MetastoreServiceGrpcClient { inner: tonic::client::Grpc, } - impl MetastoreServiceClient { + impl MetastoreServiceGrpcClient { /// Attempt to create a new client by connecting to a given endpoint. pub async fn connect(dst: D) -> Result where @@ -551,7 +3738,7 @@ pub mod metastore_service_client { Ok(Self::new(conn)) } } - impl MetastoreServiceClient + impl MetastoreServiceGrpcClient where T: tonic::client::GrpcService, T::Error: Into, @@ -569,7 +3756,7 @@ pub mod metastore_service_client { pub fn with_interceptor( inner: T, interceptor: F, - ) -> MetastoreServiceClient> + ) -> MetastoreServiceGrpcClient> where F: tonic::service::Interceptor, T::ResponseBody: Default, @@ -583,7 +3770,7 @@ pub mod metastore_service_client { http::Request, >>::Error: Into + Send + Sync, { - MetastoreServiceClient::new(InterceptedService::new(inner, interceptor)) + MetastoreServiceGrpcClient::new(InterceptedService::new(inner, interceptor)) } /// Compress requests with the given encoding. /// @@ -635,15 +3822,12 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/create_index", + "/quickwit.metastore.MetastoreService/CreateIndex", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "create_index", - ), + GrpcMethod::new("quickwit.metastore.MetastoreService", "CreateIndex"), ); self.inner.unary(req, path, codec).await } @@ -666,24 +3850,24 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/index_metadata", + "/quickwit.metastore.MetastoreService/IndexMetadata", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "index_metadata", + "IndexMetadata", ), ); self.inner.unary(req, path, codec).await } /// Gets an indexes metadatas. - pub async fn list_indexes_metadatas( + pub async fn list_indexes_metadata( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -697,14 +3881,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_indexes_metadatas", + "/quickwit.metastore.MetastoreService/ListIndexesMetadata", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "list_indexes_metadatas", + "ListIndexesMetadata", ), ); self.inner.unary(req, path, codec).await @@ -725,46 +3909,12 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/delete_index", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "delete_index", - ), - ); - self.inner.unary(req, path, codec).await - } - /// Gets all splits from index. - pub async fn list_all_splits( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_all_splits", + "/quickwit.metastore.MetastoreService/DeleteIndex", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "list_all_splits", - ), + GrpcMethod::new("quickwit.metastore.MetastoreService", "DeleteIndex"), ); self.inner.unary(req, path, codec).await } @@ -787,12 +3937,12 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_splits", + "/quickwit.metastore.MetastoreService/ListSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new("quickwit.metastore.MetastoreService", "list_splits"), + GrpcMethod::new("quickwit.metastore.MetastoreService", "ListSplits"), ); self.inner.unary(req, path, codec).await } @@ -812,15 +3962,12 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/stage_splits", + "/quickwit.metastore.MetastoreService/StageSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new( - "quickwit.metastore.MetastoreService", - "stage_splits", - ), + GrpcMethod::new("quickwit.metastore.MetastoreService", "StageSplits"), ); self.inner.unary(req, path, codec).await } @@ -840,14 +3987,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/publish_splits", + "/quickwit.metastore.MetastoreService/PublishSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "publish_splits", + "PublishSplits", ), ); self.inner.unary(req, path, codec).await @@ -868,14 +4015,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/mark_splits_for_deletion", + "/quickwit.metastore.MetastoreService/MarkSplitsForDeletion", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "mark_splits_for_deletion", + "MarkSplitsForDeletion", ), ); self.inner.unary(req, path, codec).await @@ -896,14 +4043,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/delete_splits", + "/quickwit.metastore.MetastoreService/DeleteSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "delete_splits", + "DeleteSplits", ), ); self.inner.unary(req, path, codec).await @@ -924,12 +4071,12 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/add_source", + "/quickwit.metastore.MetastoreService/AddSource", ); let mut req = request.into_request(); req.extensions_mut() .insert( - GrpcMethod::new("quickwit.metastore.MetastoreService", "add_source"), + GrpcMethod::new("quickwit.metastore.MetastoreService", "AddSource"), ); self.inner.unary(req, path, codec).await } @@ -949,14 +4096,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/toggle_source", + "/quickwit.metastore.MetastoreService/ToggleSource", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "toggle_source", + "ToggleSource", ), ); self.inner.unary(req, path, codec).await @@ -977,14 +4124,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/delete_source", + "/quickwit.metastore.MetastoreService/DeleteSource", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "delete_source", + "DeleteSource", ), ); self.inner.unary(req, path, codec).await @@ -1005,14 +4152,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/reset_source_checkpoint", + "/quickwit.metastore.MetastoreService/ResetSourceCheckpoint", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "reset_source_checkpoint", + "ResetSourceCheckpoint", ), ); self.inner.unary(req, path, codec).await @@ -1036,14 +4183,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/last_delete_opstamp", + "/quickwit.metastore.MetastoreService/LastDeleteOpstamp", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "last_delete_opstamp", + "LastDeleteOpstamp", ), ); self.inner.unary(req, path, codec).await @@ -1064,14 +4211,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/create_delete_task", + "/quickwit.metastore.MetastoreService/CreateDeleteTask", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "create_delete_task", + "CreateDeleteTask", ), ); self.inner.unary(req, path, codec).await @@ -1095,14 +4242,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/update_splits_delete_opstamp", + "/quickwit.metastore.MetastoreService/UpdateSplitsDeleteOpstamp", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "update_splits_delete_opstamp", + "UpdateSplitsDeleteOpstamp", ), ); self.inner.unary(req, path, codec).await @@ -1126,14 +4273,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_delete_tasks", + "/quickwit.metastore.MetastoreService/ListDeleteTasks", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "list_delete_tasks", + "ListDeleteTasks", ), ); self.inner.unary(req, path, codec).await @@ -1157,14 +4304,14 @@ pub mod metastore_service_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/list_stale_splits", + "/quickwit.metastore.MetastoreService/ListStaleSplits", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.metastore.MetastoreService", - "list_stale_splits", + "ListStaleSplits", ), ); self.inner.unary(req, path, codec).await @@ -1316,12 +4463,12 @@ pub mod metastore_service_client { } } /// Generated server implementations. -pub mod metastore_service_server { +pub mod metastore_service_grpc_server { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] use tonic::codegen::*; - /// Generated trait containing gRPC methods that should be implemented for use with MetastoreServiceServer. + /// Generated trait containing gRPC methods that should be implemented for use with MetastoreServiceGrpcServer. #[async_trait] - pub trait MetastoreService: Send + Sync + 'static { + pub trait MetastoreServiceGrpc: Send + Sync + 'static { /// Creates an index. async fn create_index( &self, @@ -1339,11 +4486,11 @@ pub mod metastore_service_server { tonic::Status, >; /// Gets an indexes metadatas. - async fn list_indexes_metadatas( + async fn list_indexes_metadata( &self, - request: tonic::Request, + request: tonic::Request, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, >; /// Deletes an index @@ -1351,14 +4498,6 @@ pub mod metastore_service_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; - /// Gets all splits from index. - async fn list_all_splits( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; /// Gets splits from index. async fn list_splits( &self, @@ -1484,7 +4623,7 @@ pub mod metastore_service_server { >; } #[derive(Debug)] - pub struct MetastoreServiceServer { + pub struct MetastoreServiceGrpcServer { inner: _Inner, accept_compression_encodings: EnabledCompressionEncodings, send_compression_encodings: EnabledCompressionEncodings, @@ -1492,7 +4631,7 @@ pub mod metastore_service_server { max_encoding_message_size: Option, } struct _Inner(Arc); - impl MetastoreServiceServer { + impl MetastoreServiceGrpcServer { pub fn new(inner: T) -> Self { Self::from_arc(Arc::new(inner)) } @@ -1544,9 +4683,10 @@ pub mod metastore_service_server { self } } - impl tonic::codegen::Service> for MetastoreServiceServer + impl tonic::codegen::Service> + for MetastoreServiceGrpcServer where - T: MetastoreService, + T: MetastoreServiceGrpc, B: Body + Send + 'static, B::Error: Into + Send + 'static, { @@ -1562,13 +4702,13 @@ pub mod metastore_service_server { fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { - "/quickwit.metastore.MetastoreService/create_index" => { + "/quickwit.metastore.MetastoreService/CreateIndex" => { #[allow(non_camel_case_types)] - struct create_indexSvc(pub Arc); + struct CreateIndexSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for create_indexSvc { + for CreateIndexSvc { type Response = super::CreateIndexResponse; type Future = BoxFuture< tonic::Response, @@ -1592,7 +4732,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = create_indexSvc(inner); + let method = CreateIndexSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1608,13 +4748,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/index_metadata" => { + "/quickwit.metastore.MetastoreService/IndexMetadata" => { #[allow(non_camel_case_types)] - struct index_metadataSvc(pub Arc); + struct IndexMetadataSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for index_metadataSvc { + for IndexMetadataSvc { type Response = super::IndexMetadataResponse; type Future = BoxFuture< tonic::Response, @@ -1638,7 +4778,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = index_metadataSvc(inner); + let method = IndexMetadataSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1654,25 +4794,25 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_indexes_metadatas" => { + "/quickwit.metastore.MetastoreService/ListIndexesMetadata" => { #[allow(non_camel_case_types)] - struct list_indexes_metadatasSvc(pub Arc); + struct ListIndexesMetadataSvc(pub Arc); impl< - T: MetastoreService, - > tonic::server::UnaryService - for list_indexes_metadatasSvc { - type Response = super::ListIndexesMetadatasResponse; + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for ListIndexesMetadataSvc { + type Response = super::ListIndexesMetadataResponse; type Future = BoxFuture< tonic::Response, tonic::Status, >; fn call( &mut self, - request: tonic::Request, + request: tonic::Request, ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { - (*inner).list_indexes_metadatas(request).await + (*inner).list_indexes_metadata(request).await }; Box::pin(fut) } @@ -1684,7 +4824,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = list_indexes_metadatasSvc(inner); + let method = ListIndexesMetadataSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1700,13 +4840,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/delete_index" => { + "/quickwit.metastore.MetastoreService/DeleteIndex" => { #[allow(non_camel_case_types)] - struct delete_indexSvc(pub Arc); + struct DeleteIndexSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for delete_indexSvc { + for DeleteIndexSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -1730,53 +4870,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = delete_indexSvc(inner); - let codec = tonic::codec::ProstCodec::default(); - let mut grpc = tonic::server::Grpc::new(codec) - .apply_compression_config( - accept_compression_encodings, - send_compression_encodings, - ) - .apply_max_message_size_config( - max_decoding_message_size, - max_encoding_message_size, - ); - let res = grpc.unary(method, req).await; - Ok(res) - }; - Box::pin(fut) - } - "/quickwit.metastore.MetastoreService/list_all_splits" => { - #[allow(non_camel_case_types)] - struct list_all_splitsSvc(pub Arc); - impl< - T: MetastoreService, - > tonic::server::UnaryService - for list_all_splitsSvc { - type Response = super::ListSplitsResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; - fn call( - &mut self, - request: tonic::Request, - ) -> Self::Future { - let inner = Arc::clone(&self.0); - let fut = async move { - (*inner).list_all_splits(request).await - }; - Box::pin(fut) - } - } - let accept_compression_encodings = self.accept_compression_encodings; - let send_compression_encodings = self.send_compression_encodings; - let max_decoding_message_size = self.max_decoding_message_size; - let max_encoding_message_size = self.max_encoding_message_size; - let inner = self.inner.clone(); - let fut = async move { - let inner = inner.0; - let method = list_all_splitsSvc(inner); + let method = DeleteIndexSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1792,13 +4886,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_splits" => { + "/quickwit.metastore.MetastoreService/ListSplits" => { #[allow(non_camel_case_types)] - struct list_splitsSvc(pub Arc); + struct ListSplitsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for list_splitsSvc { + for ListSplitsSvc { type Response = super::ListSplitsResponse; type Future = BoxFuture< tonic::Response, @@ -1820,7 +4914,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = list_splitsSvc(inner); + let method = ListSplitsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1836,13 +4930,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/stage_splits" => { + "/quickwit.metastore.MetastoreService/StageSplits" => { #[allow(non_camel_case_types)] - struct stage_splitsSvc(pub Arc); + struct StageSplitsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for stage_splitsSvc { + for StageSplitsSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -1866,7 +4960,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = stage_splitsSvc(inner); + let method = StageSplitsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1882,13 +4976,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/publish_splits" => { + "/quickwit.metastore.MetastoreService/PublishSplits" => { #[allow(non_camel_case_types)] - struct publish_splitsSvc(pub Arc); + struct PublishSplitsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for publish_splitsSvc { + for PublishSplitsSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -1912,7 +5006,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = publish_splitsSvc(inner); + let method = PublishSplitsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1928,13 +5022,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/mark_splits_for_deletion" => { + "/quickwit.metastore.MetastoreService/MarkSplitsForDeletion" => { #[allow(non_camel_case_types)] - struct mark_splits_for_deletionSvc(pub Arc); + struct MarkSplitsForDeletionSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for mark_splits_for_deletionSvc { + for MarkSplitsForDeletionSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -1958,7 +5052,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = mark_splits_for_deletionSvc(inner); + let method = MarkSplitsForDeletionSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -1974,13 +5068,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/delete_splits" => { + "/quickwit.metastore.MetastoreService/DeleteSplits" => { #[allow(non_camel_case_types)] - struct delete_splitsSvc(pub Arc); + struct DeleteSplitsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for delete_splitsSvc { + for DeleteSplitsSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -2004,7 +5098,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = delete_splitsSvc(inner); + let method = DeleteSplitsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2020,13 +5114,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/add_source" => { + "/quickwit.metastore.MetastoreService/AddSource" => { #[allow(non_camel_case_types)] - struct add_sourceSvc(pub Arc); + struct AddSourceSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for add_sourceSvc { + for AddSourceSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -2048,7 +5142,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = add_sourceSvc(inner); + let method = AddSourceSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2064,13 +5158,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/toggle_source" => { + "/quickwit.metastore.MetastoreService/ToggleSource" => { #[allow(non_camel_case_types)] - struct toggle_sourceSvc(pub Arc); + struct ToggleSourceSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for toggle_sourceSvc { + for ToggleSourceSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -2094,7 +5188,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = toggle_sourceSvc(inner); + let method = ToggleSourceSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2110,13 +5204,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/delete_source" => { + "/quickwit.metastore.MetastoreService/DeleteSource" => { #[allow(non_camel_case_types)] - struct delete_sourceSvc(pub Arc); + struct DeleteSourceSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for delete_sourceSvc { + for DeleteSourceSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -2140,7 +5234,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = delete_sourceSvc(inner); + let method = DeleteSourceSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2156,13 +5250,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/reset_source_checkpoint" => { + "/quickwit.metastore.MetastoreService/ResetSourceCheckpoint" => { #[allow(non_camel_case_types)] - struct reset_source_checkpointSvc(pub Arc); + struct ResetSourceCheckpointSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for reset_source_checkpointSvc { + for ResetSourceCheckpointSvc { type Response = super::EmptyResponse; type Future = BoxFuture< tonic::Response, @@ -2186,7 +5280,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = reset_source_checkpointSvc(inner); + let method = ResetSourceCheckpointSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2202,13 +5296,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/last_delete_opstamp" => { + "/quickwit.metastore.MetastoreService/LastDeleteOpstamp" => { #[allow(non_camel_case_types)] - struct last_delete_opstampSvc(pub Arc); + struct LastDeleteOpstampSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for last_delete_opstampSvc { + for LastDeleteOpstampSvc { type Response = super::LastDeleteOpstampResponse; type Future = BoxFuture< tonic::Response, @@ -2232,7 +5326,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = last_delete_opstampSvc(inner); + let method = LastDeleteOpstampSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2248,13 +5342,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/create_delete_task" => { + "/quickwit.metastore.MetastoreService/CreateDeleteTask" => { #[allow(non_camel_case_types)] - struct create_delete_taskSvc(pub Arc); + struct CreateDeleteTaskSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for create_delete_taskSvc { + for CreateDeleteTaskSvc { type Response = super::DeleteTask; type Future = BoxFuture< tonic::Response, @@ -2278,7 +5372,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = create_delete_taskSvc(inner); + let method = CreateDeleteTaskSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2294,16 +5388,16 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/update_splits_delete_opstamp" => { + "/quickwit.metastore.MetastoreService/UpdateSplitsDeleteOpstamp" => { #[allow(non_camel_case_types)] - struct update_splits_delete_opstampSvc( + struct UpdateSplitsDeleteOpstampSvc( pub Arc, ); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService< super::UpdateSplitsDeleteOpstampRequest, - > for update_splits_delete_opstampSvc { + > for UpdateSplitsDeleteOpstampSvc { type Response = super::UpdateSplitsDeleteOpstampResponse; type Future = BoxFuture< tonic::Response, @@ -2329,7 +5423,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = update_splits_delete_opstampSvc(inner); + let method = UpdateSplitsDeleteOpstampSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2345,13 +5439,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_delete_tasks" => { + "/quickwit.metastore.MetastoreService/ListDeleteTasks" => { #[allow(non_camel_case_types)] - struct list_delete_tasksSvc(pub Arc); + struct ListDeleteTasksSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for list_delete_tasksSvc { + for ListDeleteTasksSvc { type Response = super::ListDeleteTasksResponse; type Future = BoxFuture< tonic::Response, @@ -2375,7 +5469,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = list_delete_tasksSvc(inner); + let method = ListDeleteTasksSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2391,13 +5485,13 @@ pub mod metastore_service_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/list_stale_splits" => { + "/quickwit.metastore.MetastoreService/ListStaleSplits" => { #[allow(non_camel_case_types)] - struct list_stale_splitsSvc(pub Arc); + struct ListStaleSplitsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService - for list_stale_splitsSvc { + for ListStaleSplitsSvc { type Response = super::ListSplitsResponse; type Future = BoxFuture< tonic::Response, @@ -2421,7 +5515,7 @@ pub mod metastore_service_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = list_stale_splitsSvc(inner); + let method = ListStaleSplitsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2439,9 +5533,9 @@ pub mod metastore_service_server { } "/quickwit.metastore.MetastoreService/OpenShards" => { #[allow(non_camel_case_types)] - struct OpenShardsSvc(pub Arc); + struct OpenShardsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService for OpenShardsSvc { type Response = super::OpenShardsResponse; @@ -2483,9 +5577,9 @@ pub mod metastore_service_server { } "/quickwit.metastore.MetastoreService/AcquireShards" => { #[allow(non_camel_case_types)] - struct AcquireShardsSvc(pub Arc); + struct AcquireShardsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService for AcquireShardsSvc { type Response = super::AcquireShardsResponse; @@ -2529,9 +5623,9 @@ pub mod metastore_service_server { } "/quickwit.metastore.MetastoreService/CloseShards" => { #[allow(non_camel_case_types)] - struct CloseShardsSvc(pub Arc); + struct CloseShardsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService for CloseShardsSvc { type Response = super::CloseShardsResponse; @@ -2575,9 +5669,9 @@ pub mod metastore_service_server { } "/quickwit.metastore.MetastoreService/DeleteShards" => { #[allow(non_camel_case_types)] - struct DeleteShardsSvc(pub Arc); + struct DeleteShardsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService for DeleteShardsSvc { type Response = super::DeleteShardsResponse; @@ -2621,9 +5715,9 @@ pub mod metastore_service_server { } "/quickwit.metastore.MetastoreService/ListShards" => { #[allow(non_camel_case_types)] - struct ListShardsSvc(pub Arc); + struct ListShardsSvc(pub Arc); impl< - T: MetastoreService, + T: MetastoreServiceGrpc, > tonic::server::UnaryService for ListShardsSvc { type Response = super::ListShardsResponse; @@ -2678,7 +5772,7 @@ pub mod metastore_service_server { } } } - impl Clone for MetastoreServiceServer { + impl Clone for MetastoreServiceGrpcServer { fn clone(&self) -> Self { let inner = self.inner.clone(); Self { @@ -2690,7 +5784,7 @@ pub mod metastore_service_server { } } } - impl Clone for _Inner { + impl Clone for _Inner { fn clone(&self) -> Self { Self(Arc::clone(&self.0)) } @@ -2700,7 +5794,8 @@ pub mod metastore_service_server { write!(f, "{:?}", self.0) } } - impl tonic::server::NamedService for MetastoreServiceServer { + impl tonic::server::NamedService + for MetastoreServiceGrpcServer { const NAME: &'static str = "quickwit.metastore.MetastoreService"; } } diff --git a/quickwit/quickwit-proto/src/metastore/events.rs b/quickwit/quickwit-proto/src/metastore/events.rs index facffdca04b..a36e51a129f 100644 --- a/quickwit/quickwit-proto/src/metastore/events.rs +++ b/quickwit/quickwit-proto/src/metastore/events.rs @@ -19,7 +19,10 @@ use quickwit_common::pubsub::Event; -use super::{CloseShardsRequest, DeleteShardsRequest, SourceType}; +use super::{ + AddSourceRequest, CloseShardsRequest, CreateIndexRequest, DeleteIndexRequest, + DeleteShardsRequest, DeleteSourceRequest, SourceType, ToggleSourceRequest, +}; use crate::{IndexUid, SourceId}; /// Delete index event. @@ -64,6 +67,10 @@ impl Event for AddSourceEvent {} impl Event for DeleteIndexEvent {} impl Event for DeleteSourceEvent {} impl Event for ToggleSourceEvent {} - impl Event for CloseShardsRequest {} impl Event for DeleteShardsRequest {} +impl Event for AddSourceRequest {} +impl Event for CreateIndexRequest {} +impl Event for DeleteIndexRequest {} +impl Event for DeleteSourceRequest {} +impl Event for ToggleSourceRequest {} diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index 2108afbabb6..4b432db9a7d 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -22,15 +22,14 @@ use std::fmt; use quickwit_common::retry::Retryable; use serde::{Deserialize, Serialize}; -use crate::{queue_id, IndexId, QueueId, ServiceError, ServiceErrorCode, SourceId, SplitId}; +use crate::{ + queue_id, IndexId, IndexUid, QueueId, ServiceError, ServiceErrorCode, SourceId, SplitId, +}; pub mod events; include!("../codegen/quickwit/quickwit.metastore.rs"); -pub use metastore_service_client::MetastoreServiceClient; -pub use metastore_service_server::{MetastoreService, MetastoreServiceServer}; - pub type MetastoreResult = Result; /// Lists the object types stored and managed by the metastore. @@ -151,6 +150,15 @@ impl From for MetastoreError { } } +impl From for MetastoreError { + fn from(status: tonic::Status) -> Self { + serde_json::from_str(status.message()).unwrap_or_else(|_| MetastoreError::Internal { + message: "failed to deserialize metastore error".to_string(), + cause: status.message().to_string(), + }) + } +} + impl From for tonic::Status { fn from(metastore_error: MetastoreError) -> Self { let grpc_code = metastore_error.error_code().to_grpc_status_code(); @@ -209,6 +217,64 @@ impl SourceType { } } +impl IndexMetadataRequest { + pub fn for_index_uid(index_uid: String) -> Self { + Self { + index_uid: Some(index_uid), + index_id: None, + } + } + + pub fn for_index_id(index_id: String) -> Self { + Self { + index_uid: None, + index_id: Some(index_id), + } + } + + /// Returns the index id either from the `index_id` or the `index_uid`. + /// If none of them is set, an error is returned. + pub fn get_index_id(&self) -> MetastoreResult { + if let Some(index_id) = &self.index_id { + Ok(index_id.to_string()) + } else if let Some(index_uid) = &self.index_uid { + let index_uid: IndexUid = index_uid.clone().into(); + Ok(index_uid.index_id().to_string()) + } else { + Err(MetastoreError::Internal { + message: "index_id or index_uid must be set".to_string(), + cause: "".to_string(), + }) + } + } +} + +impl MarkSplitsForDeletionRequest { + pub fn new(index_uid: String, split_ids: Vec) -> Self { + Self { + index_uid, + split_ids, + } + } +} + +impl LastDeleteOpstampResponse { + pub fn new(last_delete_opstamp: u64) -> Self { + Self { + last_delete_opstamp, + } + } +} + +impl ListDeleteTasksRequest { + pub fn new(index_uid: String, opstamp_start: u64) -> Self { + Self { + index_uid, + opstamp_start, + } + } +} + impl CloseShardsSuccess { pub fn queue_id(&self) -> QueueId { queue_id(&self.index_uid, &self.source_id, self.shard_id) diff --git a/quickwit/quickwit-search/Cargo.toml b/quickwit/quickwit-search/Cargo.toml index 1d90d78f973..090db6354cd 100644 --- a/quickwit/quickwit-search/Cargo.toml +++ b/quickwit/quickwit-search/Cargo.toml @@ -61,6 +61,7 @@ typetag = { workspace = true } quickwit-indexing = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } quickwit-storage = { workspace = true, features = ["testsuite"] } [features] diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 9b955b60161..7b17fe122b9 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -49,6 +49,7 @@ pub use collector::QuickwitAggregations; use metrics::SEARCH_METRICS; use quickwit_common::tower::Pool; use quickwit_doc_mapper::DocMapper; +use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService, MetastoreServiceClient}; use tantivy::schema::NamedFieldDocument; /// Refer to this as `crate::Result`. @@ -60,7 +61,9 @@ use std::sync::Arc; pub use find_trace_ids_collector::FindTraceIdsCollector; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::tag_pruning::TagFilterAst; -use quickwit_metastore::{ListSplitsQuery, Metastore, SplitMetadata, SplitState}; +use quickwit_metastore::{ + ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, SplitMetadata, SplitState, +}; use quickwit_proto::search::{PartialHit, SearchRequest, SearchResponse, SplitIdAndFooterOffsets}; use quickwit_proto::IndexUid; use quickwit_storage::StorageResolver; @@ -168,7 +171,7 @@ async fn list_relevant_splits( start_timestamp: Option, end_timestamp: Option, tags_filter_opt: Option, - metastore: &dyn Metastore, + metastore: &mut MetastoreServiceClient, ) -> crate::Result> { let mut query = ListSplitsQuery::try_from_index_uids(index_uids)?.with_split_state(SplitState::Published); @@ -182,11 +185,15 @@ async fn list_relevant_splits( if let Some(tags_filter) = tags_filter_opt { query = query.with_tags_filter(tags_filter); } - let splits = metastore.list_splits(query).await?; - Ok(splits + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let splits_metadata: Vec = metastore + .list_splits(list_splits_request) + .await? + .deserialize_splits()? .into_iter() .map(|split| split.split_metadata) - .collect::>()) + .collect(); + Ok(splits_metadata) } /// Converts a Tantivy `NamedFieldDocument` into a json string using the @@ -207,7 +214,7 @@ fn convert_document_to_json_string( /// Starts a search node, aka a `searcher`. pub async fn start_searcher_service( - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, search_job_placer: SearchJobPlacer, searcher_context: Arc, @@ -226,7 +233,7 @@ pub async fn start_searcher_service( /// See also `[distributed_search]`. pub async fn single_node_search( search_request: SearchRequest, - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, ) -> crate::Result { let socket_addr = SocketAddr::new(Ipv4Addr::new(127, 0, 0, 1).into(), 7280u16); @@ -247,7 +254,7 @@ pub async fn single_node_search( root_search( &searcher_context, search_request, - &*metastore, + metastore, &cluster_client, ) .await diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index e95739e5333..13af3c21303 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -29,7 +29,15 @@ use quickwit_common::PrettySample; use quickwit_config::{build_doc_mapper, IndexConfig}; use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; use quickwit_doc_mapper::{DocMapper, DYNAMIC_FIELD_NAME}; -use quickwit_metastore::{IndexMetadata, ListIndexesQuery, Metastore, SplitMetadata}; +use quickwit_metastore::{ + IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataRequestExt, + ListIndexesMetadataResponseExt, ListIndexesQuery, ListSplitsRequestExt, ListSplitsResponseExt, + SplitMetadata, +}; +use quickwit_proto::metastore::{ + IndexMetadataRequest, ListIndexesMetadataRequest, ListSplitsRequest, MetastoreService, + MetastoreServiceClient, +}; use quickwit_proto::search::{ FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, ListTermsRequest, ListTermsResponse, PartialHit, @@ -690,16 +698,18 @@ fn finalize_aggregation_if_any( pub async fn root_search( searcher_context: &SearcherContext, mut search_request: SearchRequest, - metastore: &dyn Metastore, + mut metastore: MetastoreServiceClient, cluster_client: &ClusterClient, ) -> crate::Result { info!(searcher_context = ?searcher_context, search_request = ?search_request); let start_instant = tokio::time::Instant::now(); + let list_indexes_metadatas_request = ListIndexesMetadataRequest::try_from_list_indexes_query( + ListIndexesQuery::IndexIdPatterns(search_request.index_id_patterns.clone()), + )?; let indexes_metadata = metastore - .list_indexes_metadatas(ListIndexesQuery::IndexIdPatterns( - search_request.index_id_patterns.clone(), - )) - .await?; + .list_indexes_metadata(list_indexes_metadatas_request) + .await? + .deserialize_indexes_metadata()?; if indexes_metadata.is_empty() { return Err(SearchError::IndexesNotFound { index_id_patterns: search_request.index_id_patterns, @@ -730,7 +740,7 @@ pub async fn root_search( search_request.start_timestamp, search_request.end_timestamp, tag_filter_ast, - metastore, + &mut metastore, ) .await?; @@ -883,14 +893,16 @@ impl<'a, 'b> QueryAstVisitor<'b> for ExtractTimestampRange<'a> { #[instrument(skip(list_terms_request, cluster_client, metastore))] pub async fn root_list_terms( list_terms_request: &ListTermsRequest, - metastore: &dyn Metastore, + mut metastore: MetastoreServiceClient, cluster_client: &ClusterClient, ) -> crate::Result { let start_instant = tokio::time::Instant::now(); - + let index_metadata_request = + IndexMetadataRequest::for_index_id(list_terms_request.index_id.clone()); let index_metadata = metastore - .index_metadata(&list_terms_request.index_id) - .await?; + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config: IndexConfig = index_metadata.into_index_config(); @@ -924,13 +936,12 @@ pub async fn root_list_terms( if let Some(end_ts) = list_terms_request.end_timestamp { query = query.with_time_range_end_lt(end_ts); } - - let split_metadatas = metastore - .list_splits(query) + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let split_metadatas: Vec = metastore + .clone() + .list_splits(list_splits_request) .await? - .into_iter() - .map(|metadata| metadata.split_metadata) - .collect::>(); + .deserialize_splits_metadata()?; let index_uri = &index_config.index_uri; @@ -1125,7 +1136,8 @@ mod tests { use quickwit_common::shared_consts::SCROLL_BATCH_LEN; use quickwit_config::{DocMapping, IndexingSettings, SearchSettings}; use quickwit_indexing::MockSplitBuilder; - use quickwit_metastore::{IndexMetadata, MockMetastore}; + use quickwit_metastore::IndexMetadata; + 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}; use tantivy::schema::{FAST, STORED, TEXT}; @@ -1362,22 +1374,30 @@ mod tests { start_offset: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query: ListIndexesQuery| Ok(vec![index_metadata.clone()])); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ - MockSplitBuilder::new("split1") - .with_index_uid(&index_uid) - .build(), - MockSplitBuilder::new("split2") - .with_index_uid(&index_uid) - .build(), - ]) - }); + mock_metastore + .expect_list_indexes_metadata() + .returning(move |_indexes_metadata_request| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); + mock_metastore + .expect_list_splits() + .returning(move |_filter| { + let splits = vec![ + MockSplitBuilder::new("split1") + .with_index_uid(&index_uid) + .build(), + MockSplitBuilder::new("split2") + .with_index_uid(&index_uid) + .build(), + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + }); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { @@ -1433,7 +1453,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(mock_metastore), &cluster_client, ) .await @@ -1451,17 +1471,25 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") - .with_index_uid(&index_uid) - .build()]) - }); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); + metastore + .expect_list_splits() + .returning(move |_list_splits_request| { + let splits = vec![MockSplitBuilder::new("split1") + .with_index_uid(&index_uid) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + }); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().returning( |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { @@ -1493,7 +1521,7 @@ mod tests { let search_response = root_search( &searcher_context, search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -1511,21 +1539,27 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -1577,7 +1611,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -1600,21 +1634,27 @@ mod tests { }], ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -1700,7 +1740,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request.clone(), - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await?; @@ -1773,21 +1813,27 @@ mod tests { }], ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( @@ -1873,7 +1919,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request.clone(), - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await?; @@ -1941,21 +1987,27 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); @@ -2037,7 +2089,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2055,21 +2107,27 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_indexes_metadata_request| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1 @@ -2161,7 +2219,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2179,17 +2237,25 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") - .with_index_uid(&index_uid) - .build()]) - }); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); + metastore + .expect_list_splits() + .returning(move |_list_splits_request| { + let splits = vec![MockSplitBuilder::new("split1") + .with_index_uid(&index_uid) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + }); let mut first_call = true; let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().times(2).returning( @@ -2232,7 +2298,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2250,16 +2316,22 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); @@ -2289,7 +2361,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await; @@ -2306,16 +2378,22 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); // Service1 - broken node. let mut mock_search_service_1 = MockSearchService::new(); @@ -2369,7 +2447,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2388,16 +2466,22 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); // Service1 - working node. @@ -2441,7 +2525,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2453,22 +2537,31 @@ mod tests { #[tokio::test] async fn test_root_search_invalid_queries() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split") - .with_index_uid(&index_uid) - .build()]) - }); + mock_metastore + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); + mock_metastore + .expect_list_splits() + .returning(move |_filter| { + let splits = vec![MockSplitBuilder::new("split") + .with_index_uid(&index_uid) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); let searcher_context = SearcherContext::for_test(); + let metastore = MetastoreServiceClient::from(mock_metastore); assert!(root_search( &searcher_context, @@ -2478,7 +2571,7 @@ mod tests { max_hits: 10, ..Default::default() }, - &metastore, + metastore.clone(), &cluster_client, ) .await @@ -2492,7 +2585,7 @@ mod tests { max_hits: 10, ..Default::default() }, - &metastore, + metastore, &cluster_client, ) .await @@ -2529,16 +2622,22 @@ mod tests { aggregation_request: Some(agg_req.to_string()), ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); @@ -2546,7 +2645,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await; @@ -2569,24 +2668,33 @@ mod tests { start_offset: 20_000, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_list_indexes_metadatas() - .returning(move |_index_ids_query| Ok(vec![index_metadata.clone()])); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") - .with_index_uid(&index_uid) - .build()]) - }); + mock_metastore + .expect_list_indexes_metadata() + .returning(move |_index_ids_query| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata.clone() + ]) + .unwrap()) + }); + mock_metastore + .expect_list_splits() + .returning(move |_filter| { + let splits = vec![MockSplitBuilder::new("split1") + .with_index_uid(&index_uid) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) + }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); + let metastore = MetastoreServiceClient::from(mock_metastore); let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + metastore.clone(), &cluster_client, ) .await; @@ -2606,7 +2714,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + metastore, &cluster_client, ) .await; @@ -2758,25 +2866,30 @@ mod tests { #[tokio::test] async fn test_root_search_with_scroll() { - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index-1", "ram:///test-index-1"); let index_uid = index_metadata.index_uid.clone(); let index_metadata_2 = IndexMetadata::for_test("test-index-2", "ram:///test-index-2"); let index_uid_2 = index_metadata_2.index_uid.clone(); metastore - .expect_list_indexes_metadatas() + .expect_list_indexes_metadata() .returning(move |_index_ids_query| { - Ok(vec![index_metadata.clone(), index_metadata_2.clone()]) + let indexes_metadata = vec![index_metadata.clone(), index_metadata_2.clone()]; + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(indexes_metadata) + .unwrap(), + ) }); metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid_2) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().times(2).returning( @@ -2860,7 +2973,7 @@ mod tests { let search_response = root_search( &searcher_context, search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await @@ -2936,7 +3049,7 @@ mod tests { max_hits: 10, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata_1 = IndexMetadata::for_test("test-index-1", "ram:///test-index-1"); let index_uid_1 = index_metadata_1.index_uid.clone(); let index_metadata_2 = @@ -2945,9 +3058,12 @@ mod tests { let index_metadata_3 = index_metadata_for_multi_indexes_test("test-index-3", "ram:///test-index-3"); let index_uid_3 = index_metadata_3.index_uid.clone(); - metastore.expect_list_indexes_metadatas().return_once( - move |index_ids_query: ListIndexesQuery| { - match index_ids_query { + metastore.expect_list_indexes_metadata().return_once( + move |list_indexes_metadata_request: ListIndexesMetadataRequest| { + let query = list_indexes_metadata_request + .deserialize_list_indexes_query() + .unwrap(); + match query { ListIndexesQuery::IndexIdPatterns(index_ids_query) => { assert_eq!(index_ids_query, vec!["test-index-*".to_string()]); } @@ -2955,12 +3071,19 @@ mod tests { panic!("Unexpected empty index_ids_query"); } } - Ok(vec![index_metadata_1, index_metadata_2, index_metadata_3]) + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + index_metadata_1, + index_metadata_2, + index_metadata_3, + ]) + .unwrap()) }, ); metastore .expect_list_splits() - .return_once(move |list_splits_query| { + .return_once(move |list_splits_request| { + let list_splits_query = + list_splits_request.deserialize_list_splits_query().unwrap(); assert!( list_splits_query.index_uids == vec![ @@ -2969,7 +3092,7 @@ mod tests { index_uid_3.clone() ] ); - Ok(vec![ + let splits = vec![ MockSplitBuilder::new("index-1-split-1") .with_index_uid(&index_uid_1) .build(), @@ -2979,7 +3102,8 @@ mod tests { MockSplitBuilder::new("index-2-split-1") .with_index_uid(&index_uid_2) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1 @@ -3027,7 +3151,7 @@ mod tests { let search_response = root_search( &SearcherContext::for_test(), search_request, - &metastore, + MetastoreServiceClient::from(metastore), &cluster_client, ) .await diff --git a/quickwit/quickwit-search/src/search_stream/leaf.rs b/quickwit/quickwit-search/src/search_stream/leaf.rs index e4d73d6cd89..40bce90f37d 100644 --- a/quickwit/quickwit-search/src/search_stream/leaf.rs +++ b/quickwit/quickwit-search/src/search_stream/leaf.rs @@ -451,6 +451,8 @@ mod tests { use itertools::Itertools; use quickwit_indexing::TestSandbox; + use quickwit_metastore::{ListSplitsRequestExt, ListSplitsResponseExt}; + use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; use quickwit_query::query_ast::qast_json_helper; use serde_json::json; use tantivy::time::{Duration, OffsetDateTime}; @@ -498,11 +500,12 @@ mod tests { }; let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) - .await?; + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) + .await? + .deserialize_splits()?; let splits_offsets = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let searcher_context = Arc::new(SearcherContext::for_test()); let mut single_node_stream = leaf_search_stream( @@ -574,11 +577,12 @@ mod tests { }; let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) - .await?; + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) + .await? + .deserialize_splits()?; let splits_offsets = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let searcher_context = Arc::new(SearcherContext::for_test()); let mut single_node_stream = leaf_search_stream( @@ -629,11 +633,12 @@ mod tests { }; let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) - .await?; + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) + .await? + .deserialize_splits()?; let splits_offsets = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let searcher_context = Arc::new(SearcherContext::for_test()); let mut single_node_stream = leaf_search_stream( @@ -717,11 +722,12 @@ mod tests { }; let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) - .await?; + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) + .await? + .deserialize_splits()?; let splits_offsets = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let searcher_context = Arc::new(SearcherContext::for_test()); let mut single_node_stream = leaf_search_stream( diff --git a/quickwit/quickwit-search/src/search_stream/root.rs b/quickwit/quickwit-search/src/search_stream/root.rs index f69f601cc6d..41daf548408 100644 --- a/quickwit/quickwit-search/src/search_stream/root.rs +++ b/quickwit/quickwit-search/src/search_stream/root.rs @@ -24,7 +24,8 @@ use futures::{StreamExt, TryStreamExt}; use quickwit_common::uri::Uri; use quickwit_config::build_doc_mapper; use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; -use quickwit_metastore::Metastore; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::search::{LeafSearchStreamRequest, SearchRequest, SearchStreamRequest}; use quickwit_query::query_ast::QueryAst; use tokio_stream::StreamMap; @@ -38,15 +39,17 @@ use crate::{list_relevant_splits, SearchError}; #[instrument(skip(metastore, cluster_client))] pub async fn root_search_stream( mut search_stream_request: SearchStreamRequest, - metastore: &dyn Metastore, + mut metastore: MetastoreServiceClient, cluster_client: ClusterClient, ) -> crate::Result>> { // TODO: building a search request should not be necessary for listing splits. // This needs some refactoring: relevant splits, metadata_map, jobs... - + let index_metadata_request = + IndexMetadataRequest::for_index_id(search_stream_request.index_id.clone()); let index_metadata = metastore - .index_metadata(&search_stream_request.index_id) - .await?; + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid = index_metadata.index_uid.clone(); let index_config = index_metadata.into_index_config(); @@ -79,7 +82,7 @@ pub async fn root_search_stream( search_request.start_timestamp, search_request.end_timestamp, tags_filter_ast, - metastore, + &mut metastore, ) .await?; @@ -130,7 +133,8 @@ fn jobs_to_leaf_request( mod tests { use quickwit_indexing::MockSplitBuilder; - use quickwit_metastore::{IndexMetadata, MockMetastore}; + use quickwit_metastore::{IndexMetadata, ListSplitsResponseExt}; + use quickwit_proto::metastore::{IndexMetadataResponse, ListSplitsResponse}; use quickwit_proto::search::OutputFormat; use quickwit_query::query_ast::qast_json_helper; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -147,16 +151,17 @@ mod tests { output_format: OutputFormat::Csv as i32, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + mock_metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + mock_metastore.expect_list_splits().returning(move |_| { + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -179,10 +184,14 @@ mod tests { let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); - let result: Vec = root_search_stream(request, &metastore, cluster_client) - .await? - .try_collect() - .await?; + let result: Vec = root_search_stream( + request, + MetastoreServiceClient::from(mock_metastore), + cluster_client, + ) + .await? + .try_collect() + .await?; assert_eq!(result.len(), 2); assert_eq!(&result[0], &b"123"[..]); assert_eq!(&result[1], &b"456"[..]); @@ -199,16 +208,17 @@ mod tests { partition_by_field: Some("timestamp".to_string()), ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split1") + mock_metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + mock_metastore.expect_list_splits().returning(move |_| { + let splits = vec![MockSplitBuilder::new("split1") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -231,7 +241,12 @@ mod tests { let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); - let stream = root_search_stream(request, &metastore, cluster_client).await?; + let stream = root_search_stream( + request, + MetastoreServiceClient::from(mock_metastore), + cluster_client, + ) + .await?; let result: Vec<_> = stream.try_collect().await?; assert_eq!(result.len(), 2); assert_eq!(&result[0], &b"123"[..]); @@ -248,21 +263,22 @@ mod tests { output_format: OutputFormat::Csv as i32, ..Default::default() }; - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + mock_metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + mock_metastore.expect_list_splits().returning(move |_| { + let splits = vec![ MockSplitBuilder::new("split1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -293,7 +309,12 @@ mod tests { let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); - let stream = root_search_stream(request, &metastore, cluster_client).await?; + let stream = root_search_stream( + request, + MetastoreServiceClient::from(mock_metastore), + cluster_client, + ) + .await?; let result: Result, SearchError> = stream.try_collect().await; assert_eq!(result.is_err(), true); assert_eq!(result.unwrap_err().to_string(), "internal error: `error`"); @@ -302,21 +323,22 @@ mod tests { #[tokio::test] async fn test_root_search_stream_with_invalid_query() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![MockSplitBuilder::new("split") + mock_metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + mock_metastore.expect_list_splits().returning(move |_| { + let splits = vec![MockSplitBuilder::new("split") .with_index_uid(&index_uid) - .build()]) + .build()]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", MockSearchService::new())]); let search_job_placer = SearchJobPlacer::new(searcher_pool); - + let metastore = MetastoreServiceClient::from(mock_metastore); assert!(root_search_stream( quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), @@ -326,7 +348,7 @@ mod tests { partition_by_field: Some("timestamp".to_string()), ..Default::default() }, - &metastore, + metastore.clone(), ClusterClient::new(search_job_placer.clone()), ) .await @@ -341,7 +363,7 @@ mod tests { partition_by_field: Some("timestamp".to_string()), ..Default::default() }, - &metastore, + metastore, ClusterClient::new(search_job_placer.clone()), ) .await diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index 6e7ca090bef..f75d200bc34 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -27,7 +27,7 @@ use bytes::Bytes; use quickwit_common::uri::Uri; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::DocMapper; -use quickwit_metastore::Metastore; +use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_proto::search::{ FetchDocsRequest, FetchDocsResponse, GetKvRequest, Hit, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, LeafSearchStreamRequest, @@ -54,7 +54,7 @@ use crate::{ #[derive(Clone)] /// The search service implementation. pub struct SearchServiceImpl { - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, cluster_client: ClusterClient, searcher_context: Arc, @@ -141,7 +141,7 @@ pub trait SearchService: 'static + Send + Sync { impl SearchServiceImpl { /// Creates a new search service. pub fn new( - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, cluster_client: ClusterClient, searcher_context: Arc, @@ -169,7 +169,7 @@ impl SearchService for SearchServiceImpl { let search_result = root_search( &self.searcher_context, search_request, - self.metastore.as_ref(), + self.metastore.clone(), &self.cluster_client, ) .await?; @@ -232,7 +232,7 @@ impl SearchService for SearchServiceImpl { ) -> crate::Result> + Send>>> { let data = root_search_stream( stream_request, - self.metastore.as_ref(), + self.metastore.clone(), self.cluster_client.clone(), ) .await?; @@ -268,7 +268,7 @@ impl SearchService for SearchServiceImpl { ) -> crate::Result { let search_result = root_list_terms( &list_terms_request, - self.metastore.as_ref(), + self.metastore.clone(), &self.cluster_client, ) .await?; diff --git a/quickwit/quickwit-search/src/tests.rs b/quickwit/quickwit-search/src/tests.rs index 4bfff29515d..6f84651a62a 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -980,7 +980,7 @@ async fn test_single_node_split_pruning_by_tags() -> anyhow::Result<()> { None, None, extract_tags_from_query(query_ast), - &*test_sandbox.metastore(), + &mut test_sandbox.metastore(), ) .await?; assert!(selected_splits.is_empty()); @@ -992,7 +992,7 @@ async fn test_single_node_split_pruning_by_tags() -> anyhow::Result<()> { None, None, extract_tags_from_query(query_ast), - &*test_sandbox.metastore(), + &mut test_sandbox.metastore(), ) .await?; assert_eq!(selected_splits.len(), 2); @@ -1004,7 +1004,7 @@ async fn test_single_node_split_pruning_by_tags() -> anyhow::Result<()> { None, None, extract_tags_from_query(query_ast), - &*test_sandbox.metastore(), + &mut test_sandbox.metastore(), ) .await?; assert_eq!(selected_splits.len(), 2); @@ -1026,12 +1026,14 @@ async fn test_single_node_split_pruning_by_tags() -> anyhow::Result<()> { async fn test_search_util(test_sandbox: &TestSandbox, query: &str) -> Vec { let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) .await + .unwrap() + .deserialize_splits() .unwrap(); let splits_offsets: Vec<_> = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let request = Arc::new(SearchRequest { index_id_patterns: vec![test_sandbox.index_uid().index_id().to_string()], @@ -1663,12 +1665,12 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { let splits = test_sandbox .metastore() - .list_all_splits(test_sandbox.index_uid()) - .await - .unwrap(); + .list_splits(ListSplitsRequest::try_from_index_uid(test_sandbox.index_uid()).unwrap()) + .await? + .deserialize_splits()?; let splits_offsets: Vec<_> = splits .into_iter() - .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) + .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); let searcher_context = Arc::new(SearcherContext::new(SearcherConfig::default(), None)); diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index a7f214cc036..28d72369c7e 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -83,5 +83,6 @@ quickwit-indexing = { workspace = true, features = ["testsuite"] } quickwit-ingest = { workspace = true, features = ["testsuite"] } quickwit-janitor = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } quickwit-search = { workspace = true, features = ["testsuite"] } quickwit-storage = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 462e468a601..919ec78871d 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -17,12 +17,13 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; - use quickwit_config::build_doc_mapper; use quickwit_janitor::error::JanitorError; -use quickwit_metastore::Metastore; -use quickwit_proto::metastore::{DeleteQuery, DeleteTask, MetastoreResult}; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::metastore::{ + DeleteQuery, DeleteTask, IndexMetadataRequest, ListDeleteTasksRequest, MetastoreResult, + MetastoreService, MetastoreServiceClient, +}; use quickwit_proto::search::SearchRequest; use quickwit_proto::IndexUid; use quickwit_query::query_ast::{query_ast_from_user_text, QueryAst}; @@ -58,13 +59,13 @@ pub struct DeleteQueryRequest { /// Delete query API handlers. pub fn delete_task_api_handlers( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { get_delete_tasks_handler(metastore.clone()).or(post_delete_tasks_handler(metastore.clone())) } pub fn get_delete_tasks_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!(String / "delete-tasks") .and(warp::get()) @@ -94,15 +95,24 @@ pub fn get_delete_tasks_handler( // `get_delete_tasks_handler` and consequently we get the mailbox in `get_delete_tasks` signature. pub async fn get_delete_tasks( index_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult> { - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; - let delete_tasks = metastore.list_delete_tasks(index_uid, 0).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + let list_delete_tasks_request = ListDeleteTasksRequest::new(index_uid.to_string(), 0); + let delete_tasks = metastore + .list_delete_tasks(list_delete_tasks_request) + .await? + .delete_tasks; Ok(delete_tasks) } pub fn post_delete_tasks_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!(String / "delete-tasks") .and(warp::body::json()) @@ -132,9 +142,13 @@ pub fn post_delete_tasks_handler( pub async fn post_delete_request( index_id: String, delete_request: DeleteQueryRequest, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> Result { - let metadata = metastore.index_metadata(&index_id).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let index_uid: IndexUid = metadata.index_uid.clone(); let query_ast = query_ast_from_user_text(&delete_request.query, Some(Vec::new())) .parse_user_query(&[]) diff --git a/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs b/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs index 4efc6426f05..fafd1c5a672 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs @@ -371,7 +371,7 @@ mod tests { async fn test_bulk_ingest_request_returns_400_if_action_is_malformed() { let config = Arc::new(NodeConfig::for_test()); let search_service = Arc::new(MockSearchService::new()); - let ingest_service = IngestServiceClient::new(IngestServiceClient::mock()); + let ingest_service = IngestServiceClient::from(IngestServiceClient::mock()); let elastic_api_handlers = elastic_api_handlers(config, search_service, ingest_service); let payload = r#" {"create": {"_index": "my-index", "_id": "1"},} diff --git a/quickwit/quickwit-serve/src/grpc.rs b/quickwit/quickwit-serve/src/grpc.rs index 468dafffb19..3820c114144 100644 --- a/quickwit/quickwit-serve/src/grpc.rs +++ b/quickwit/quickwit-serve/src/grpc.rs @@ -24,11 +24,9 @@ use std::sync::Arc; use quickwit_common::tower::BoxFutureInfaillible; use quickwit_config::service::QuickwitService; use quickwit_jaeger::JaegerService; -use quickwit_metastore::GrpcMetastoreAdapter; use quickwit_opentelemetry::otlp::{OtlpGrpcLogsService, OtlpGrpcTracesService}; use quickwit_proto::indexing::IndexingServiceClient; use quickwit_proto::jaeger::storage::v1::span_reader_plugin_server::SpanReaderPluginServer; -use quickwit_proto::metastore::MetastoreServiceServer; use quickwit_proto::opentelemetry::proto::collector::logs::v1::logs_service_server::LogsServiceServer; use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_server::TraceServiceServer; use quickwit_proto::search::search_service_server::SearchServiceServer; @@ -52,8 +50,7 @@ pub(crate) async fn start_grpc_server( // Mount gRPC metastore service if `QuickwitService::Metastore` is enabled on node. let metastore_grpc_service = if let Some(metastore_server) = &services.metastore_server_opt { enabled_grpc_services.insert("metastore"); - let grpc_metastore_adapter = GrpcMetastoreAdapter::from(metastore_server.clone()); - Some(MetastoreServiceServer::new(grpc_metastore_adapter)) + Some(metastore_server.as_grpc_service()) } else { None }; diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index e87d155fbfc..09832196145 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -29,9 +29,15 @@ use quickwit_config::{ use quickwit_doc_mapper::{analyze_text, TokenizerConfig}; use quickwit_index_management::{IndexService, IndexServiceError}; use quickwit_metastore::{ - IndexMetadata, ListIndexesQuery, ListSplitsQuery, Metastore, Split, SplitInfo, SplitState, + IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataRequestExt, + ListIndexesMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, + Split, SplitInfo, SplitState, +}; +use quickwit_proto::metastore::{ + DeleteSourceRequest, EntityKind, IndexMetadataRequest, ListIndexesMetadataRequest, + ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, + MetastoreService, MetastoreServiceClient, ResetSourceCheckpointRequest, ToggleSourceRequest, }; -use quickwit_proto::metastore::{EntityKind, MetastoreError, MetastoreResult}; use quickwit_proto::IndexUid; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; @@ -120,7 +126,7 @@ pub fn config_format_filter() -> impl Filter, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String) .and(warp::get()) @@ -132,14 +138,19 @@ fn get_index_metadata_handler( async fn get_index_metadata( index_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, "get-index-metadata"); - metastore.index_metadata(&index_id).await + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; + Ok(index_metadata) } fn get_indexes_metadatas_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes") .and(warp::get()) @@ -179,11 +190,19 @@ struct IndexStats { /// Describes an index. async fn describe_index( index_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult { - let index_metadata = metastore.index_metadata(&index_id).await?; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; let query = ListSplitsQuery::for_index(index_metadata.index_uid.clone()); - let splits = metastore.list_splits(query).await?; + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let splits = metastore + .list_splits(list_splits_request) + .await? + .deserialize_splits()?; let published_splits: Vec = splits .into_iter() .filter(|split| split.split_state == SplitState::Published) @@ -226,7 +245,7 @@ async fn describe_index( } fn describe_index_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "describe") .and(warp::get()) @@ -280,9 +299,14 @@ pub struct ListSplitsQueryParams { async fn list_splits( index_id: String, list_split_query: ListSplitsQueryParams, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult> { - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; info!(index_id = %index_id, list_split_query = ?list_split_query, "get-splits"); let mut query = ListSplitsQuery::for_index(index_uid); if let Some(split_states) = list_split_query.split_states { @@ -297,11 +321,16 @@ async fn list_splits( if let Some(end_created_timestamp) = list_split_query.end_create_timestamp { query = query.with_create_timestamp_lt(end_created_timestamp); } - metastore.list_splits(query).await + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(query)?; + let splits = metastore + .list_splits(list_splits_request) + .await? + .deserialize_splits()?; + Ok(splits) } fn list_splits_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "splits") .and(warp::get()) @@ -334,22 +363,30 @@ struct SplitsForDeletion { async fn mark_splits_for_deletion( index_id: String, splits_for_deletion: SplitsForDeletion, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; info!(index_id = %index_id, splits_ids = ?splits_for_deletion.split_ids, "mark-splits-for-deletion"); - let split_ids: Vec<&str> = splits_for_deletion + let split_ids: Vec = splits_for_deletion .split_ids .iter() - .map(|split_id| split_id.as_ref()) + .map(|split_id| split_id.to_string()) .collect(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid.to_string(), split_ids.clone()); metastore - .mark_splits_for_deletion(index_uid, &split_ids) - .await + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await?; + Ok(()) } fn mark_splits_for_deletion_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "splits" / "mark-for-deletion") .and(warp::put()) @@ -371,12 +408,13 @@ fn mark_splits_for_deletion_handler( )] /// Gets indexes metadata. async fn get_indexes_metadatas( - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult> { info!("get-indexes-metadatas"); metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .and_then(|response| response.deserialize_indexes_metadata()) } #[derive(Deserialize, utoipa::IntoParams, utoipa::ToSchema)] @@ -421,7 +459,7 @@ async fn create_index( create_index_query_params: CreateIndexQueryParams, config_format: ConfigFormat, index_config_bytes: Bytes, - index_service: IndexService, + mut index_service: IndexService, node_config: Arc, ) -> Result { let index_config = quickwit_config::load_index_config_from_user_config( @@ -462,7 +500,7 @@ fn clear_index_handler( /// configuration. (See also, `delete-index`). async fn clear_index( index_id: String, - index_service: IndexService, + mut index_service: IndexService, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, "clear-index"); index_service.clear_index(&index_id).await @@ -504,7 +542,7 @@ fn delete_index_handler( async fn delete_index( index_id: String, delete_index_query_param: DeleteIndexQueryParam, - index_service: IndexService, + mut index_service: IndexService, ) -> Result, IndexServiceError> { info!(index_id = %index_id, dry_run = delete_index_query_param.dry_run, "delete-index"); index_service @@ -544,7 +582,7 @@ async fn create_source( index_id: String, config_format: ConfigFormat, source_config_bytes: Bytes, - index_service: IndexService, + mut index_service: IndexService, ) -> Result { let source_config: SourceConfig = load_source_config_from_user_config(config_format, &source_config_bytes) @@ -556,17 +594,19 @@ async fn create_source( .to_string(), )); } + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_uid: IndexUid = index_service .metastore() - .index_metadata(&index_id) + .index_metadata(index_metadata_request) .await? + .deserialize_index_metadata()? .index_uid; info!(index_id = %index_id, source_id = %source_config.source_id, "create-source"); index_service.create_source(index_uid, source_config).await } fn get_source_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "sources" / String) .and(warp::get()) @@ -579,12 +619,14 @@ fn get_source_handler( async fn get_source( index_id: String, source_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, source_id = %source_id, "get-source"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let source_config = metastore - .index_metadata(&index_id) + .index_metadata(index_metadata_request) .await? + .deserialize_index_metadata()? .sources .remove(&source_id) .ok_or({ @@ -597,7 +639,7 @@ async fn get_source( } fn reset_source_checkpoint_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "sources" / String / "reset-checkpoint") .and(warp::put()) @@ -623,17 +665,27 @@ fn reset_source_checkpoint_handler( async fn reset_source_checkpoint( index_id: String, source_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; + let index_metadata_resquest = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_resquest) + .await? + .deserialize_index_metadata()? + .index_uid; info!(index_id = %index_id, source_id = %source_id, "reset-checkpoint"); + let reset_source_checkpoint_request = ResetSourceCheckpointRequest { + index_uid: index_uid.to_string(), + source_id: source_id.clone(), + }; metastore - .reset_source_checkpoint(index_uid, &source_id) - .await + .reset_source_checkpoint(reset_source_checkpoint_request) + .await?; + Ok(()) } fn toggle_source_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "sources" / String / "toggle") .and(warp::put()) @@ -668,24 +720,32 @@ async fn toggle_source( index_id: String, source_id: String, toggle_source: ToggleSource, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, source_id = %source_id, enable = toggle_source.enable, "toggle-source"); - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; if [CLI_INGEST_SOURCE_ID, INGEST_API_SOURCE_ID].contains(&source_id.as_str()) { return Err(IndexServiceError::OperationNotAllowed(format!( "source `{source_id}` is managed by Quickwit, you cannot enable or disable a source \ managed by Quickwit" ))); } - metastore - .toggle_source(index_uid, &source_id, toggle_source.enable) - .await?; + let toggle_source_request = ToggleSourceRequest { + index_uid: index_uid.to_string(), + source_id: source_id.clone(), + enable: toggle_source.enable, + }; + metastore.toggle_source(toggle_source_request).await?; Ok(()) } fn delete_source_handler( - metastore: Arc, + metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String / "sources" / String) .and(warp::delete()) @@ -711,17 +771,26 @@ fn delete_source_handler( async fn delete_source( index_id: String, source_id: String, - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, source_id = %source_id, "delete-source"); - let index_uid: IndexUid = metastore.index_metadata(&index_id).await?.index_uid; + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; if [INGEST_API_SOURCE_ID, CLI_INGEST_SOURCE_ID].contains(&source_id.as_str()) { return Err(IndexServiceError::OperationNotAllowed(format!( "source `{source_id}` is managed by Quickwit, you cannot delete a source managed by \ Quickwit" ))); } - metastore.delete_source(index_uid, &source_id).await?; + let delete_source_request = DeleteSourceRequest { + index_uid: index_uid.to_string(), + source_id: source_id.clone(), + }; + metastore.delete_source(delete_source_request).await?; Ok(()) } @@ -774,8 +843,11 @@ mod tests { use quickwit_common::uri::Uri; use quickwit_config::{SourceParams, VecSourceParams}; use quickwit_indexing::{mock_split, MockSplitBuilder}; - use quickwit_metastore::{metastore_for_test, IndexMetadata, ListIndexesQuery, MockMetastore}; - use quickwit_proto::metastore::SourceType; + use quickwit_metastore::{metastore_for_test, IndexMetadata}; + use quickwit_proto::metastore::{ + EmptyResponse, IndexMetadataResponse, ListIndexesMetadataResponse, ListSplitsResponse, + MetastoreServiceClient, SourceType, + }; use quickwit_storage::StorageResolver; use serde_json::Value as JsonValue; @@ -784,16 +856,20 @@ mod tests { #[tokio::test] async fn test_get_index() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore - .expect_index_metadata() - .return_once(|_index_id: &str| { - Ok(IndexMetadata::for_test( + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore.expect_index_metadata().return_once(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( "test-index", "ram:///indexes/test-index", )) - }); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + .unwrap(), + ) + }); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -830,17 +906,20 @@ mod tests { #[tokio::test] async fn test_get_splits() { - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("quickwit-demo-index", "ram:///indexes/quickwit-demo-index"); let index_uid = index_metadata.index_uid.clone(); metastore .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())) + .returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }) .times(2); metastore .expect_list_splits() - .returning(move |list_split_query: ListSplitsQuery| { + .returning(move |list_splits_request: ListSplitsRequest| { + let list_split_query = list_splits_request.deserialize_list_splits_query().unwrap(); if list_split_query.index_uids.contains(&index_uid) && list_split_query.split_states == vec![SplitState::Published, SplitState::Staged] @@ -848,9 +927,10 @@ mod tests { && list_split_query.time_range.end == Bound::Excluded(20) && list_split_query.create_timestamp.end == Bound::Excluded(2) { - return Ok(vec![MockSplitBuilder::new("split_1") + let splits = vec![MockSplitBuilder::new("split_1") .with_index_uid(&index_uid) - .build()]); + .build()]; + return Ok(ListSplitsResponse::try_from_splits(splits).unwrap()); } Err(MetastoreError::Internal { message: "".to_string(), @@ -858,7 +938,10 @@ mod tests { }) }) .times(2); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let index_service = IndexService::new( + MetastoreServiceClient::from(metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -895,13 +978,15 @@ mod tests { #[tokio::test] async fn test_describe_index() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("quickwit-demo-index", "ram:///indexes/quickwit-demo-index"); let index_uid = index_metadata.index_uid.clone(); - metastore + mock_metastore .expect_index_metadata() - .return_once(move |_index_id: &str| Ok(index_metadata)); + .return_once(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap()) + }); let split_1 = MockSplitBuilder::new("split_1") .with_index_uid(&index_uid) .build(); @@ -913,19 +998,21 @@ mod tests { split_1_time_range.start() - 10, split_1_time_range.end() + 10, )); - metastore + mock_metastore .expect_list_splits() - .return_once(move |list_split_query: ListSplitsQuery| { - if list_split_query.index_uids.contains(&index_uid) { - return Ok(vec![split_1, split_2]); - } - Err(MetastoreError::Internal { - message: "".to_string(), - cause: "".to_string(), - }) + .withf(move |list_split_request| -> bool { + let list_split_query = list_split_request.deserialize_list_splits_query().unwrap(); + list_split_query.index_uids.contains(&index_uid) + }) + .return_once(move |_| { + let splits = vec![split_1, split_2]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -954,29 +1041,35 @@ mod tests { #[tokio::test] async fn test_get_all_splits() { - let mut metastore = MockMetastore::new(); + let mut mock_metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("quickwit-demo-index", "ram:///indexes/quickwit-demo-index"); let index_uid = index_metadata.index_uid.clone(); - metastore + mock_metastore .expect_index_metadata() - .return_once(move |_index_id: &str| Ok(index_metadata)); - metastore - .expect_list_splits() - .return_once(move |list_split_query: ListSplitsQuery| { + .return_once(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata).unwrap()) + }); + mock_metastore.expect_list_splits().return_once( + move |list_split_request: ListSplitsRequest| { + let list_split_query = list_split_request.deserialize_list_splits_query().unwrap(); if list_split_query.index_uids.contains(&index_uid) && list_split_query.split_states.is_empty() && list_split_query.time_range.is_unbounded() && list_split_query.create_timestamp.is_unbounded() { - return Ok(Vec::new()); + return Ok(ListSplitsResponse::empty()); } Err(MetastoreError::Internal { message: "".to_string(), cause: "".to_string(), }) - }); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + }, + ); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -989,31 +1082,41 @@ mod tests { #[tokio::test] async fn test_mark_splits_for_deletion() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_index_metadata() - .returning(|_index_id: &str| { - Ok(IndexMetadata::for_test( - "quickwit-demo-index", - "ram:///indexes/quickwit-demo-index", - )) + .returning(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( + "quickwit-demo-index", + "ram:///indexes/quickwit-demo-index", + )) + .unwrap(), + ) }) .times(2); - metastore + mock_metastore .expect_mark_splits_for_deletion() - .returning(|index_uid: IndexUid, split_ids: &[&str]| { - if index_uid.index_id() == "quickwit-demo-index" - && split_ids == ["split-1", "split-2"] - { - return Ok(()); - } - Err(MetastoreError::Internal { - message: "".to_string(), - cause: "".to_string(), - }) - }) + .returning( + |mark_splits_for_deletion_request: MarkSplitsForDeletionRequest| { + let split_ids = mark_splits_for_deletion_request.split_ids; + let index_uid: IndexUid = mark_splits_for_deletion_request.index_uid.into(); + if index_uid.index_id() == "quickwit-demo-index" + && split_ids == ["split-1", "split-2"] + { + return Ok(EmptyResponse {}); + } + Err(MetastoreError::Internal { + message: "".to_string(), + cause: "".to_string(), + }) + }, + ) .times(2); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1038,16 +1141,21 @@ mod tests { #[tokio::test] async fn test_get_list_indexes() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore.expect_list_indexes_metadatas().return_once( - |_list_indexes_query: ListIndexesQuery| { - Ok(vec![IndexMetadata::for_test( - "test-index", - "ram:///indexes/test-index", - )]) - }, + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore + .expect_list_indexes_metadata() + .return_once(|_list_indexes_request| { + let index_metadata = + IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(vec![index_metadata]) + .unwrap(), + ) + }); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), ); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1073,28 +1181,32 @@ mod tests { #[tokio::test] async fn test_clear_index() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore - .expect_index_metadata() - .return_once(|_index_id: &str| { - Ok(IndexMetadata::for_test( + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore.expect_index_metadata().return_once(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( "quickwit-demo-index", "file:///path/to/index/quickwit-demo-index", )) - }); - metastore - .expect_list_all_splits() - .return_once(|_| Ok(vec![mock_split("split_1")])); - metastore + .unwrap(), + ) + }); + mock_metastore.expect_list_splits().return_once(|_| { + Ok(ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap()) + }); + mock_metastore .expect_mark_splits_for_deletion() - .return_once(|_index_id: IndexUid, _splits: &[&str]| Ok(())); - metastore + .return_once(|_| Ok(EmptyResponse {})); + mock_metastore .expect_delete_splits() - .return_once(|_index_id: IndexUid, _splits: &[&str]| Ok(())); - metastore + .return_once(|_| Ok(EmptyResponse {})); + mock_metastore .expect_reset_source_checkpoint() - .return_once(|_index_id: IndexUid, _source_id: &str| Ok(())); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + .return_once(|_| Ok(EmptyResponse {})); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1109,33 +1221,38 @@ mod tests { #[tokio::test] async fn test_delete_index() { - let mut metastore = MockMetastore::new(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_index_metadata() - .returning(|_index_id: &str| { - Ok(IndexMetadata::for_test( - "quickwit-demo-index", - "file:///path/to/index/quickwit-demo-index", - )) + .returning(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( + "quickwit-demo-index", + "file:///path/to/index/quickwit-demo-index", + )) + .unwrap(), + ) }) .times(2); - metastore - .expect_list_all_splits() - .return_once(|_| Ok(vec![mock_split("split_1")])); - metastore + mock_metastore .expect_list_splits() - .returning(|_| Ok(vec![mock_split("split_1")])) - .times(2); - metastore + .returning(|_| { + Ok(ListSplitsResponse::try_from_splits(vec![mock_split("split_1")]).unwrap()) + }) + .times(3); + mock_metastore .expect_mark_splits_for_deletion() - .return_once(|_index_uid: IndexUid, _splits: &[&str]| Ok(())); - metastore + .return_once(|_| Ok(EmptyResponse {})); + mock_metastore .expect_delete_splits() - .return_once(|_index_uid: IndexUid, _splits: &[&str]| Ok(())); - metastore + .return_once(|_| Ok(EmptyResponse {})); + mock_metastore .expect_delete_index() - .return_once(|_index_uid: IndexUid| Ok(())); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + .return_once(|_| Ok(EmptyResponse {})); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1228,7 +1345,7 @@ mod tests { #[tokio::test] async fn test_create_delete_index_and_source() { - let metastore = metastore_for_test(); + let mut metastore = metastore_for_test(); let index_service = IndexService::new(metastore.clone(), StorageResolver::unconfigured()); let mut node_config = NodeConfig::for_test(); node_config.default_index_root_uri = @@ -1272,7 +1389,12 @@ mod tests { assert_eq!(resp.status(), 200); // Check that the source has been added to index metadata. - let index_metadata = metastore.index_metadata("hdfs-logs").await.unwrap(); + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); assert!(index_metadata.sources.contains_key("vec-source")); let source_config = index_metadata.sources.get("vec-source").unwrap(); assert_eq!(source_config.source_type(), SourceType::Vec); @@ -1293,7 +1415,12 @@ mod tests { .reply(&index_management_handler) .await; assert_eq!(resp.status(), 200); - let index_metadata = metastore.index_metadata("hdfs-logs").await.unwrap(); + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); assert!(!index_metadata.sources.contains_key("file-source")); // Check cannot delete source managed by Quickwit. @@ -1331,8 +1458,10 @@ mod tests { .await; assert_eq!(resp.status(), 200); let indexes = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await + .unwrap() + .deserialize_indexes_metadata() .unwrap(); assert!(indexes.is_empty()); } @@ -1347,7 +1476,8 @@ mod tests { let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)) .recover(recover_fn); - let source_config_body = r#"{"version": "0.6", "source_id": "file-source", "source_type": "file", "params": {"filepath": "FILEPATH"}}"#; + let source_config_body = r#"{"version": "0.6", "source_id": "file-source", "source_type": + "file", "params": {"filepath": "FILEPATH"}}"#; let resp = warp::test::request() .path("/indexes/hdfs-logs/sources") .method("POST") @@ -1459,8 +1589,11 @@ mod tests { #[tokio::test] async fn test_create_index_with_bad_config() -> anyhow::Result<()> { - let metastore = MockMetastore::new(); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let metastore = MetastoreServiceClient::mock(); + let index_service = IndexService::new( + MetastoreServiceClient::from(metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1507,7 +1640,11 @@ mod tests { .path("/indexes/my-index/sources") .method("POST") .json(&true) - .body(r#"{"version": "0.6", "source_id": "pulsar-source", "desired_num_pipelines": 2, "source_type": "pulsar", "params": {"topics": ["my-topic"], "address": "pulsar://localhost:6650" }}"#) + .body( + r#"{"version": "0.6", "source_id": "pulsar-source", + "desired_num_pipelines": 2, "source_type": "pulsar", "params": {"topics": ["my-topic"], + "address": "pulsar://localhost:6650" }}"#, + ) .reply(&index_management_handler) .await; assert_eq!(resp.status(), 400); @@ -1521,28 +1658,35 @@ mod tests { #[tokio::test] async fn test_delete_non_existing_source() { - let mut metastore = MockMetastore::new(); - metastore - .expect_index_metadata() - .return_once(|_index_id: &str| { - Ok(IndexMetadata::for_test( + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore.expect_index_metadata().return_once(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( "quickwit-demo-index", "file:///path/to/index/quickwit-demo-index", )) - }); - metastore - .expect_index_exists() - .return_once(|index_id: &str| Ok(index_id == "quickwit-demo-index")); - metastore - .expect_delete_source() - .return_once(|index_uid, source_id| { + .unwrap(), + ) + }); + // TODO + // metastore + // .expect_index_exists() + // .return_once(|index_id: &str| Ok(index_id == "quickwit-demo-index")); + mock_metastore.expect_delete_source().return_once( + |delete_source_request: DeleteSourceRequest| { + let source_id = delete_source_request.source_id; + let index_uid: IndexUid = delete_source_request.index_uid.into(); assert_eq!(index_uid.index_id(), "quickwit-demo-index"); Err(MetastoreError::NotFound(EntityKind::Source { index_id: "quickwit-demo-index".to_string(), source_id: source_id.to_string(), })) - }); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + }, + ); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1556,29 +1700,41 @@ mod tests { #[tokio::test] async fn test_source_reset_checkpoint() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_index_metadata() - .returning(|_index_id: &str| { - Ok(IndexMetadata::for_test( - "quickwit-demo-index", - "file:///path/to/index/quickwit-demo-index", - )) + .returning(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( + "quickwit-demo-index", + "file:///path/to/index/quickwit-demo-index", + )) + .unwrap(), + ) }) .times(2); - metastore + mock_metastore .expect_reset_source_checkpoint() - .returning(|index_uid: IndexUid, source_id: &str| { - if index_uid.index_id() == "quickwit-demo-index" && source_id == "source-to-reset" { - return Ok(()); - } - Err(MetastoreError::Internal { - message: "".to_string(), - cause: "".to_string(), - }) - }) + .returning( + |reset_source_checkpoint_request: ResetSourceCheckpointRequest| { + let source_id = reset_source_checkpoint_request.source_id; + let index_uid: IndexUid = reset_source_checkpoint_request.index_uid.into(); + if index_uid.index_id() == "quickwit-demo-index" + && source_id == "source-to-reset" + { + return Ok(EmptyResponse {}); + } + Err(MetastoreError::Internal { + message: "".to_string(), + cause: "".to_string(), + }) + }, + ) .times(2); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1599,23 +1755,29 @@ mod tests { #[tokio::test] async fn test_source_toggle() -> anyhow::Result<()> { - let mut metastore = MockMetastore::new(); - metastore + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore .expect_index_metadata() - .returning(|_index_id: &str| { - Ok(IndexMetadata::for_test( - "quickwit-demo-index", - "file:///path/to/index/quickwit-demo-index", - )) + .returning(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( + "quickwit-demo-index", + "file:///path/to/index/quickwit-demo-index", + )) + .unwrap(), + ) }) .times(3); - metastore.expect_toggle_source().return_once( - |index_uid: IndexUid, source_id: &str, enable: bool| { + mock_metastore.expect_toggle_source().return_once( + |toggle_source_request: ToggleSourceRequest| { + let source_id = toggle_source_request.source_id; + let index_uid: IndexUid = toggle_source_request.index_uid.into(); + let enable = toggle_source_request.enable; if index_uid.index_id() == "quickwit-demo-index" && source_id == "source-to-toggle" && enable { - return Ok(()); + return Ok(EmptyResponse {}); } Err(MetastoreError::Internal { message: "".to_string(), @@ -1623,7 +1785,10 @@ mod tests { }) }, ); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + let index_service = IndexService::new( + MetastoreServiceClient::from(mock_metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1671,16 +1836,20 @@ mod tests { #[tokio::test] async fn test_analyze_request() { - let mut metastore = MockMetastore::new(); - metastore - .expect_index_metadata() - .return_once(|_index_id: &str| { - Ok(IndexMetadata::for_test( + let mut metastore = MetastoreServiceClient::mock(); + metastore.expect_index_metadata().return_once(|_| { + Ok( + IndexMetadataResponse::try_from_index_metadata(IndexMetadata::for_test( "test-index", "ram:///indexes/test-index", )) - }); - let index_service = IndexService::new(Arc::new(metastore), StorageResolver::unconfigured()); + .unwrap(), + ) + }); + let index_service = IndexService::new( + MetastoreServiceClient::from(metastore), + StorageResolver::unconfigured(), + ); let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); @@ -1688,7 +1857,10 @@ mod tests { .path("/analyze") .method("POST") .json(&true) - .body(r#"{"type": "ngram", "min_gram": 3, "max_gram": 3, "text": "Hel", "filters": ["lower_caser"]}"#) + .body( + r#"{"type": "ngram", "min_gram": 3, "max_gram": 3, "text": "Hel", "filters": + ["lower_caser"]}"#, + ) .reply(&index_management_handler) .await; assert_eq!(resp.status(), 200); diff --git a/quickwit/quickwit-serve/src/ingest_metastore.rs b/quickwit/quickwit-serve/src/ingest_metastore.rs index 8a876e6891b..44323d52b26 100644 --- a/quickwit/quickwit-serve/src/ingest_metastore.rs +++ b/quickwit/quickwit-serve/src/ingest_metastore.rs @@ -17,13 +17,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::sync::Arc; - use async_trait::async_trait; -use quickwit_metastore::Metastore; use quickwit_proto::ingest::{IngestV2Error, IngestV2Result}; use quickwit_proto::metastore::{ CloseShardsRequest, CloseShardsResponse, DeleteShardsRequest, DeleteShardsResponse, + MetastoreService, MetastoreServiceClient, }; // TODO: Remove when the metastore is code generated in `quickwit-proto`. @@ -32,11 +30,11 @@ use quickwit_proto::metastore::{ /// where it is defined for more details about why this is required. #[derive(Clone)] pub(crate) struct IngestMetastoreImpl { - metastore: Arc, + metastore: MetastoreServiceClient, } impl IngestMetastoreImpl { - pub fn new(metastore: Arc) -> Self { + pub fn new(metastore: MetastoreServiceClient) -> Self { Self { metastore } } } @@ -48,6 +46,7 @@ impl quickwit_ingest::IngestMetastore for IngestMetastoreImpl { request: CloseShardsRequest, ) -> IngestV2Result { self.metastore + .clone() .close_shards(request) .await .map_err(|error| IngestV2Error::Internal(error.to_string())) @@ -58,6 +57,7 @@ impl quickwit_ingest::IngestMetastore for IngestMetastoreImpl { request: DeleteShardsRequest, ) -> IngestV2Result { self.metastore + .clone() .delete_shards(request) .await .map_err(|error| IngestV2Error::Internal(error.to_string())) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index fe0385328c4..548ae4e49d3 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -60,7 +60,7 @@ use quickwit_common::pubsub::{EventBroker, EventSubscriptionHandle}; use quickwit_common::runtimes::RuntimesConfig; use quickwit_common::tower::{ BalanceChannel, BoxFutureInfaillible, BufferLayer, Change, ConstantRate, EstimateRateLayer, - RateLimitLayer, SmaRateEstimator, + EventListenerLayer, RateLimitLayer, RetryLayer, RetryPolicy, SmaRateEstimator, }; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; @@ -75,8 +75,8 @@ use quickwit_ingest::{ }; use quickwit_janitor::{start_janitor_service, JanitorService}; use quickwit_metastore::{ - ControlPlaneMetastore, ListIndexesQuery, Metastore, MetastoreEventPublisher, - MetastoreGrpcClient, MetastoreResolver, RetryingMetastore, + ControlPlaneMetastore, ListIndexesMetadataRequestExt, ListIndexesMetadataResponseExt, + MetastoreResolver, }; use quickwit_opentelemetry::otlp::{OtlpGrpcLogsService, OtlpGrpcTracesService}; use quickwit_proto::control_plane::ControlPlaneServiceClient; @@ -84,7 +84,8 @@ use quickwit_proto::indexing::IndexingServiceClient; use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::ingest::router::IngestRouterServiceClient; use quickwit_proto::metastore::{ - CloseShardsRequest, DeleteShardsRequest, EntityKind, MetastoreError, + CloseShardsRequest, DeleteShardsRequest, EntityKind, ListIndexesMetadataRequest, + MetastoreError, MetastoreService, MetastoreServiceClient, }; use quickwit_proto::search::ReportSplitsRequest; use quickwit_proto::NodeId; @@ -116,8 +117,8 @@ const READINESS_REPORTING_INTERVAL: Duration = if cfg!(any(test, feature = "test struct QuickwitServices { pub node_config: Arc, pub cluster: Cluster, - pub metastore_server_opt: Option>, - pub metastore_client: Arc, + pub metastore_server_opt: Option, + pub metastore_client: MetastoreServiceClient, pub control_plane_service: ControlPlaneServiceClient, pub index_manager: IndexManager, pub indexing_service_opt: Option>, @@ -209,7 +210,7 @@ async fn start_ingest_client_if_needed( async fn start_control_plane_if_needed( node_config: &NodeConfig, cluster: &Cluster, - metastore_client: &Arc, + metastore_client: &MetastoreServiceClient, universe: &Universe, indexer_pool: &IndexerPool, ingester_pool: &IngesterPool, @@ -267,41 +268,50 @@ pub async fn serve_quickwit( let universe = Universe::new(); // Instantiate a metastore "server" if the `metastore` role is enabled on the node. - let metastore_server_opt: Option> = + let metastore_server_opt: Option = if node_config.is_service_enabled(QuickwitService::Metastore) { - let metastore = metastore_resolver + let metastore: MetastoreServiceClient = metastore_resolver .resolve(&node_config.metastore_uri) .await?; - let metastore = MetastoreEventPublisher::new(metastore.clone(), event_broker.clone()); - Some(Arc::new(metastore)) + let broker_layer = EventListenerLayer::new(event_broker.clone()); + let metastore = MetastoreServiceClient::tower() + .create_index_layer(broker_layer.clone()) + .delete_index_layer(broker_layer.clone()) + .add_source_layer(broker_layer.clone()) + .delete_source_layer(broker_layer.clone()) + .toggle_source_layer(broker_layer) + .build(metastore); + Some(metastore) } else { None }; // Instantiate a metastore client, either local if available or remote otherwise. - let metastore_client: Arc = if let Some(metastore_server) = &metastore_server_opt - { - metastore_server.clone() - } else { - // Wait for a metastore service to be available for at most 10 seconds. - if cluster - .wait_for_ready_members(has_node_with_metastore_service, Duration::from_secs(10)) - .await - .is_err() - { - error!("No metastore service found among cluster members, stopping server."); - anyhow::bail!( - "failed to start server: no metastore service was found among cluster members. \ - try running Quickwit with additional metastore service `quickwit run --service \ - metastore`" - ) - } - let balance_channel = - balance_channel_for_service(&cluster, QuickwitService::Metastore).await; - let grpc_metastore_client = - MetastoreGrpcClient::from_balance_channel(balance_channel).await?; - let metastore_client = RetryingMetastore::new(Box::new(grpc_metastore_client)); - Arc::new(metastore_client) - }; + let metastore_client: MetastoreServiceClient = + if let Some(metastore_server) = &metastore_server_opt { + metastore_server.clone() + } else { + // Wait for a metastore service to be available for at most 10 seconds. + if cluster + .wait_for_ready_members(has_node_with_metastore_service, Duration::from_secs(10)) + .await + .is_err() + { + error!("No metastore service found among cluster members, stopping server."); + anyhow::bail!( + "failed to start server: no metastore service was found among cluster \ + members. try running Quickwit with additional metastore service `quickwit \ + run --service metastore`" + ) + } + + let balance_channel = + balance_channel_for_service(&cluster, QuickwitService::Metastore).await; + let metastore_client = MetastoreServiceClient::from_balance_channel(balance_channel); + let retry_layer = RetryLayer::new(RetryPolicy::default()); + MetastoreServiceClient::tower() + .shared_layer(retry_layer) + .build(metastore_client) + }; // Instantiate a control plane server if the `control-plane` role is enabled on the node. // Otherwise, instantiate a control plane client. @@ -322,8 +332,7 @@ pub async fn serve_quickwit( &control_plane_service, ); - // Set up the "control plane proxy" for the metastore. - let metastore_through_control_plane: Arc = Arc::new(ControlPlaneMetastore::new( + let metastore_through_control_plane = MetastoreServiceClient::new(ControlPlaneMetastore::new( control_plane_service.clone(), metastore_client.clone(), )); @@ -372,7 +381,7 @@ pub async fn serve_quickwit( // Any node can serve index management requests (create/update/delete index, add/remove source, // etc.), so we always instantiate an index manager. - let index_manager = IndexManager::new( + let mut index_manager = IndexManager::new( metastore_through_control_plane.clone(), storage_resolver.clone(), ); @@ -577,7 +586,7 @@ fn setup_control_plane_event_subscriptions( async fn setup_ingest_v2( config: &NodeConfig, cluster: &Cluster, - metastore: Arc, + metastore: MetastoreServiceClient, control_plane: ControlPlaneServiceClient, ingester_pool: IngesterPool, ) -> anyhow::Result<(IngestRouterServiceClient, Option)> { @@ -649,7 +658,7 @@ async fn setup_ingest_v2( async fn setup_searcher( cluster_change_stream: impl Stream + Send + 'static, - metastore: Arc, + metastore: MetastoreServiceClient, storage_resolver: StorageResolver, searcher_context: Arc, ) -> anyhow::Result<(SearchJobPlacer, Arc)> { @@ -698,7 +707,7 @@ async fn setup_control_plane( self_node_id: String, indexer_pool: IndexerPool, ingester_pool: IngesterPool, - metastore: Arc, + metastore: MetastoreServiceClient, replication_factor: usize, ) -> anyhow::Result> { let self_node_id: NodeId = self_node_id.into(); @@ -789,7 +798,7 @@ fn with_arg(arg: T) -> impl Filter, + mut metastore: MetastoreServiceClient, grpc_readiness_signal_rx: oneshot::Receiver<()>, rest_readiness_signal_rx: oneshot::Receiver<()>, ) { @@ -812,11 +821,11 @@ async fn node_readiness_reporting_task( let node_ready = match metastore.check_connectivity().await { Ok(()) => { - debug!(metastore_uri=%metastore.uri(), "Metastore service is available."); + debug!(metastore_endpoints=?metastore.endpoints(), "Metastore service is available."); true } Err(error) => { - warn!(metastore_uri=%metastore.uri(), error=?error, "Metastore service is unavailable."); + warn!(metastore_endpoints=?metastore.endpoints(), error=?error, "Metastore service is unavailable."); false } }; @@ -829,20 +838,25 @@ async fn node_readiness_reporting_task( async fn check_cluster_configuration( services: &HashSet, peer_seeds: &[String], - metastore: Arc, + mut metastore: MetastoreServiceClient, ) -> anyhow::Result<()> { if !services.contains(&QuickwitService::Metastore) || peer_seeds.is_empty() { return Ok(()); } - if !metastore.uri().protocol().is_database() { + if metastore + .endpoints() + .iter() + .any(|uri| !uri.protocol().is_database()) + { warn!( - metastore_uri=%metastore.uri(), - "Using a file-backed metastore in cluster mode is not recommended for production use. Running multiple file-backed metastores simultaneously can lead to data loss." - ); + metastore_endpoints=?metastore.endpoints(), + "Using a file-backed metastore in cluster mode is not recommended for production use. + Running multiple file-backed metastores simultaneously can lead to data loss."); } let file_backed_indexes = metastore - .list_indexes_metadatas(ListIndexesQuery::All) + .list_indexes_metadata(ListIndexesMetadataRequest::all()) .await? + .deserialize_indexes_metadata()? .into_iter() .filter(|index_metadata| index_metadata.index_uri().protocol().is_file_storage()) .collect::>(); @@ -870,8 +884,9 @@ mod tests { use quickwit_cluster::{create_cluster_for_test, ClusterNode}; use quickwit_common::uri::Uri; use quickwit_config::SearcherConfig; - use quickwit_metastore::{metastore_for_test, IndexMetadata, ListIndexesQuery, MockMetastore}; + use quickwit_metastore::{metastore_for_test, IndexMetadata}; use quickwit_proto::indexing::IndexingTask; + use quickwit_proto::metastore::ListIndexesMetadataResponse; use quickwit_search::Job; use tokio::sync::{mpsc, watch}; use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream}; @@ -882,24 +897,27 @@ mod tests { async fn test_check_cluster_configuration() { let services = HashSet::from_iter([QuickwitService::Metastore]); let peer_seeds = ["192.168.0.12:7280".to_string()]; - let mut metastore = MockMetastore::new(); - - metastore - .expect_uri() - .return_const(Uri::for_test("file:///qwdata/indexes")); - - metastore.expect_list_indexes_metadatas().return_once( - |_list_indexes_query: ListIndexesQuery| { - Ok(vec![IndexMetadata::for_test( - "test-index", - "file:///qwdata/indexes/test-index", - )]) - }, - ); + let mut mock_metastore = MetastoreServiceClient::mock(); + + mock_metastore + .expect_endpoints() + .return_const(vec![Uri::for_test("file:///qwdata/indexes")]); + mock_metastore + .expect_list_indexes_metadata() + .return_once(|_| { + Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(vec![ + IndexMetadata::for_test("test-index", "file:///qwdata/indexes/test-index"), + ]) + .unwrap()) + }); - check_cluster_configuration(&services, &peer_seeds, Arc::new(metastore)) - .await - .unwrap(); + check_cluster_configuration( + &services, + &peer_seeds, + MetastoreServiceClient::from(mock_metastore), + ) + .await + .unwrap(); } #[tokio::test] @@ -909,19 +927,21 @@ mod tests { .await .unwrap(); let (metastore_readiness_tx, metastore_readiness_rx) = watch::channel(false); - let mut metastore = MockMetastore::new(); - metastore.expect_check_connectivity().returning(move || { - if *metastore_readiness_rx.borrow() { - Ok(()) - } else { - Err(anyhow::anyhow!("Metastore not ready")) - } - }); + let mut mock_metastore = MetastoreServiceClient::mock(); + mock_metastore + .expect_check_connectivity() + .returning(move || { + if *metastore_readiness_rx.borrow() { + Ok(()) + } else { + Err(anyhow::anyhow!("Metastore not ready")) + } + }); let (grpc_readiness_trigger_tx, grpc_readiness_signal_rx) = oneshot::channel(); let (rest_readiness_trigger_tx, rest_readiness_signal_rx) = oneshot::channel(); tokio::spawn(node_readiness_reporting_task( cluster.clone(), - Arc::new(metastore), + MetastoreServiceClient::from(mock_metastore), grpc_readiness_signal_rx, rest_readiness_signal_rx, )); diff --git a/quickwit/quickwit-serve/src/search_api/mod.rs b/quickwit/quickwit-serve/src/search_api/mod.rs index be368b94913..b77cc5d919c 100644 --- a/quickwit/quickwit-serve/src/search_api/mod.rs +++ b/quickwit/quickwit-serve/src/search_api/mod.rs @@ -34,7 +34,10 @@ mod tests { use futures::TryStreamExt; use quickwit_indexing::MockSplitBuilder; - use quickwit_metastore::{IndexMetadata, MockMetastore}; + use quickwit_metastore::{IndexMetadata, IndexMetadataResponseExt, ListSplitsResponseExt}; + use quickwit_proto::metastore::{ + IndexMetadataResponse, ListSplitsResponse, MetastoreServiceClient, + }; use quickwit_proto::search::search_service_server::SearchServiceServer; use quickwit_proto::search::OutputFormat; use quickwit_proto::tonic; @@ -76,21 +79,22 @@ mod tests { output_format: OutputFormat::Csv as i32, partition_by_field: None, }; - let mut metastore = MockMetastore::new(); + let mut metastore = MetastoreServiceClient::mock(); let index_metadata = IndexMetadata::for_test("test-index", "ram:///indexes/test-index"); let index_uid = index_metadata.index_uid.clone(); - metastore - .expect_index_metadata() - .returning(move |_index_id: &str| Ok(index_metadata.clone())); - metastore.expect_list_splits().returning(move |_filter| { - Ok(vec![ + metastore.expect_index_metadata().returning(move |_| { + Ok(IndexMetadataResponse::try_from_index_metadata(index_metadata.clone()).unwrap()) + }); + metastore.expect_list_splits().returning(move |_| { + let splits = vec![ MockSplitBuilder::new("split_1") .with_index_uid(&index_uid) .build(), MockSplitBuilder::new("split_2") .with_index_uid(&index_uid) .build(), - ]) + ]; + Ok(ListSplitsResponse::try_from_splits(splits).unwrap()) }); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); @@ -125,7 +129,12 @@ mod tests { searcher_pool.insert(grpc_addr, create_search_client_from_grpc_addr(grpc_addr)); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); - let stream = root_search_stream(request, &metastore, cluster_client).await?; + let stream = root_search_stream( + request, + MetastoreServiceClient::from(metastore), + cluster_client, + ) + .await?; let search_stream_result: Result, SearchError> = stream.try_collect().await; let search_error = search_stream_result.unwrap_err(); assert_eq!(