From 0ebb7dd971c8ffbb7212451279d734dde7030f5c Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Fri, 27 Oct 2023 15:52:19 +0900 Subject: [PATCH 01/12] Slightly better logging (#4035) --- quickwit/quickwit-indexing/src/actors/merge_pipeline.rs | 7 ++++--- quickwit/quickwit-search/src/root.rs | 4 ++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 7b483ad3207..7c1314bd199 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -217,7 +217,7 @@ impl MergePipeline { pipeline_ord=%self.params.pipeline_id.pipeline_ord, root_dir=%self.params.indexing_directory.path().display(), merge_policy=?self.params.merge_policy, - "Spawning merge pipeline.", + "spawn merge pipeline", ); let query = ListSplitsQuery::for_index(self.params.pipeline_id.index_uid.clone()) .with_split_state(SplitState::Published) @@ -229,9 +229,10 @@ impl MergePipeline { .deserialize_splits_metadata()?; info!( - "splits_metadata.len() = {}", - published_splits_metadata.len() + num_splits = published_splits_metadata.len(), + "loaded list of published splits" ); + // Merge publisher let merge_publisher = Publisher::new( PublisherType::MergePublisher, diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 703973138e2..367a3416400 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -615,7 +615,7 @@ pub(crate) async fn fetch_docs_phase( /// 2. Merges the search results. /// 3. Sends fetch docs requests to multiple leaf nodes. /// 4. Builds the response with docs and returns. -#[instrument(skip_all)] +#[instrument(skip_all, fields(num_splits=%split_metadatas.len()))] async fn root_search_aux( searcher_context: &SearcherContext, indexes_metas_for_leaf_search: &IndexesMetasForLeafSearch, @@ -623,7 +623,7 @@ async fn root_search_aux( split_metadatas: Vec, cluster_client: &ClusterClient, ) -> crate::Result { - info!(split_metadatas = ?PrettySample::new(&split_metadatas, 5)); + debug!(split_metadatas = ?PrettySample::new(&split_metadatas, 5)); let (first_phase_result, scroll_key_and_start_offset_opt): ( LeafSearchResponse, Option, From 67e3ee94e70027d39851b15e3bfd91d84128d3fa Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 00:08:42 +0900 Subject: [PATCH 02/12] Exposing pipeline metrics (#4026) * Exposing pipeline metrics * Apply suggestions from code review Co-authored-by: Adrien Guillo * Moved metrics to proto/indexing * Removing obsolete pipeline metrics keys. Updated chitchat, and using the new iter_prefix method. --------- Co-authored-by: Adrien Guillo --- quickwit/Cargo.lock | 5 +- quickwit/Cargo.toml | 2 +- quickwit/quickwit-actors/src/actor_context.rs | 3 +- quickwit/quickwit-cluster/src/cluster.rs | 59 ++++++++++++------- quickwit/quickwit-cluster/src/member.rs | 9 +-- quickwit/quickwit-cluster/src/node.rs | 2 +- .../quickwit-indexing/src/actors/indexer.rs | 39 +++++++++++- .../src/actors/indexing_pipeline.rs | 12 ++-- .../src/actors/indexing_service.rs | 16 ++++- .../src/models/indexing_statistics.rs | 3 + quickwit/quickwit-proto/src/indexing/mod.rs | 31 ++++++++-- 11 files changed, 141 insertions(+), 40 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 2310258e725..de278b96d92 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -1182,8 +1182,9 @@ dependencies = [ [[package]] name = "chitchat" -version = "0.5.0" -source = "git+https://github.com/quickwit-oss/chitchat?rev=bc29598#bc295980ac2e00f389dfa7e87cf6dc7995061206" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6e5c649e5309f040f18aa0d660e63fe913d3d642e4771dd3cb66515ad75d239d" dependencies = [ "anyhow", "async-trait", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index cee159ab6b3..261add432f8 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -49,7 +49,7 @@ base64 = "0.21" byte-unit = { version = "4", default-features = false, features = ["serde", "std"] } bytes = { version = "1", features = ["serde"] } bytestring = "1.3.0" -chitchat = { git = "https://github.com/quickwit-oss/chitchat", rev = "bc29598" } +chitchat = "0.6" chrono = { version = "0.4.23", default-features = false, features = ["clock", "std"] } clap = { version = "4.4.1", features = ["env", "string"] } colored = "2.0.0" diff --git a/quickwit/quickwit-actors/src/actor_context.rs b/quickwit/quickwit-actors/src/actor_context.rs index b0201d92a9d..b4391ee7574 100644 --- a/quickwit/quickwit-actors/src/actor_context.rs +++ b/quickwit/quickwit-actors/src/actor_context.rs @@ -214,7 +214,8 @@ impl ActorContext { self.observe_enqueued.swap(true, Ordering::Relaxed) } - pub(crate) fn observe(&self, actor: &mut A) -> A::ObservableState { + /// Updates the observable state of the actor. + pub fn observe(&self, actor: &mut A) -> A::ObservableState { let obs_state = actor.observable_state(); self.inner.observe_enqueued.store(false, Ordering::Relaxed); let _ = self.observable_state_tx.send(obs_state.clone()); diff --git a/quickwit/quickwit-cluster/src/cluster.rs b/quickwit/quickwit-cluster/src/cluster.rs index df5ec0c3f4e..8abe200c64b 100644 --- a/quickwit/quickwit-cluster/src/cluster.rs +++ b/quickwit/quickwit-cluster/src/cluster.rs @@ -17,7 +17,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::collections::{BTreeMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt::Debug; use std::net::SocketAddr; use std::sync::Arc; @@ -31,7 +31,7 @@ use chitchat::{ }; use futures::Stream; use itertools::Itertools; -use quickwit_proto::indexing::IndexingTask; +use quickwit_proto::indexing::{IndexingPipelineId, IndexingTask, PipelineMetrics}; use quickwit_proto::types::NodeId; use serde::{Deserialize, Serialize}; use tokio::sync::{mpsc, watch, Mutex, RwLock}; @@ -43,8 +43,8 @@ use tracing::{info, warn}; use crate::change::{compute_cluster_change_events, ClusterChange}; use crate::member::{ build_cluster_member, ClusterMember, NodeStateExt, ENABLED_SERVICES_KEY, - GRPC_ADVERTISE_ADDR_KEY, INDEXING_TASK_PREFIX, READINESS_KEY, READINESS_VALUE_NOT_READY, - READINESS_VALUE_READY, + GRPC_ADVERTISE_ADDR_KEY, INDEXING_TASK_PREFIX, PIPELINE_METRICS_PREFIX, READINESS_KEY, + READINESS_VALUE_NOT_READY, READINESS_VALUE_READY, }; use crate::ClusterNode; @@ -70,7 +70,7 @@ pub struct Cluster { } impl Debug for Cluster { - fn fmt(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, formatter: &mut std::fmt::Formatter) -> std::fmt::Result { formatter .debug_struct("Cluster") .field("cluster_id", &self.cluster_id) @@ -301,9 +301,34 @@ impl Cluster { tokio::time::sleep(GOSSIP_INTERVAL * 2).await; } + /// This exposes in chitchat some metrics about the CPU usage of cooperative pipelines. + /// The metrics are exposed as follows: + /// Key: pipeline_metrics:: + /// Value: 179‰,76MB/s + pub async fn update_self_node_pipeline_metrics( + &self, + pipeline_metrics: &HashMap<&IndexingPipelineId, PipelineMetrics>, + ) { + let chitchat = self.chitchat().await; + let mut chitchat_guard = chitchat.lock().await; + let node_state = chitchat_guard.self_node_state(); + let mut current_metrics_keys: HashSet = node_state + .iter_prefix(PIPELINE_METRICS_PREFIX) + .map(|(key, _)| key.to_string()) + .collect(); + for (pipeline_id, metrics) in pipeline_metrics { + let key = format!("{PIPELINE_METRICS_PREFIX}{pipeline_id}"); + current_metrics_keys.remove(&key); + node_state.set(key, metrics.to_string()); + } + for obsolete_task_key in current_metrics_keys { + node_state.mark_for_deletion(&obsolete_task_key); + } + } + /// Updates indexing tasks in chitchat state. /// Tasks are grouped by (index_id, source_id), each group is stored in a key as follows: - /// - key: `{INDEXING_TASK_PREFIX}{INDEXING_TASK_SEPARATOR}{index_id}{INDEXING_TASK_SEPARATOR}{source_id}` + /// - key: `{INDEXING_TASK_PREFIX}{index_id}{INDEXING_TASK_SEPARATOR}{source_id}` /// - value: Number of indexing tasks in the group. /// Keys present in chitchat state but not in the given `indexing_tasks` are marked for /// deletion. @@ -313,24 +338,20 @@ impl Cluster { ) -> anyhow::Result<()> { let chitchat = self.chitchat().await; let mut chitchat_guard = chitchat.lock().await; - let mut current_indexing_tasks_keys: HashSet<_> = chitchat_guard - .self_node_state() - .key_values(|key, _| key.starts_with(INDEXING_TASK_PREFIX)) + let node_state = chitchat_guard.self_node_state(); + let mut current_indexing_tasks_keys: HashSet = node_state + .iter_prefix(INDEXING_TASK_PREFIX) .map(|(key, _)| key.to_string()) .collect(); for (indexing_task, indexing_tasks_group) in indexing_tasks.iter().group_by(|&task| task).into_iter() { - let key = format!("{INDEXING_TASK_PREFIX}:{}", indexing_task.to_string()); + let key = format!("{INDEXING_TASK_PREFIX}{indexing_task}"); current_indexing_tasks_keys.remove(&key); - chitchat_guard - .self_node_state() - .set(key, indexing_tasks_group.count().to_string()); + node_state.set(key, indexing_tasks_group.count().to_string()); } for obsolete_task_key in current_indexing_tasks_keys { - chitchat_guard - .self_node_state() - .mark_for_deletion(&obsolete_task_key); + node_state.mark_for_deletion(&obsolete_task_key); } Ok(()) } @@ -949,13 +970,11 @@ mod tests { let chitchat_handle = node.inner.read().await.chitchat_handle.chitchat(); let mut chitchat_guard = chitchat_handle.lock().await; chitchat_guard.self_node_state().set( - format!( - "{INDEXING_TASK_PREFIX}:my_good_index:my_source:11111111111111111111111111" - ), + format!("{INDEXING_TASK_PREFIX}my_good_index:my_source:11111111111111111111111111"), "2".to_string(), ); chitchat_guard.self_node_state().set( - format!("{INDEXING_TASK_PREFIX}:my_bad_index:my_source:11111111111111111111111111"), + format!("{INDEXING_TASK_PREFIX}my_bad_index:my_source:11111111111111111111111111"), "malformatted value".to_string(), ); } diff --git a/quickwit/quickwit-cluster/src/member.rs b/quickwit/quickwit-cluster/src/member.rs index 4f9c8aec83e..728546134e7 100644 --- a/quickwit/quickwit-cluster/src/member.rs +++ b/quickwit/quickwit-cluster/src/member.rs @@ -32,10 +32,11 @@ use crate::{GenerationId, QuickwitService}; // Keys used to store member's data in chitchat state. pub(crate) const GRPC_ADVERTISE_ADDR_KEY: &str = "grpc_advertise_addr"; pub(crate) const ENABLED_SERVICES_KEY: &str = "enabled_services"; +pub(crate) const PIPELINE_METRICS_PREFIX: &str = "pipeline_metrics:"; + // An indexing task key is formatted as // `{INDEXING_TASK_PREFIX}{INDEXING_TASK_SEPARATOR}{index_id}{INDEXING_TASK_SEPARATOR}{source_id}`. -pub(crate) const INDEXING_TASK_PREFIX: &str = "indexing_task"; -pub(crate) const INDEXING_TASK_SEPARATOR: char = ':'; +pub(crate) const INDEXING_TASK_PREFIX: &str = "indexing_task:"; // Readiness key and values used to store node's readiness in Chitchat state. pub(crate) const READINESS_KEY: &str = "readiness"; @@ -163,7 +164,7 @@ pub(crate) fn build_cluster_member( // Parses indexing task key into the IndexingTask. fn parse_indexing_task_key(key: &str) -> anyhow::Result { - let (_prefix, reminder) = key.split_once(INDEXING_TASK_SEPARATOR).ok_or_else(|| { + let reminder = key.strip_prefix(INDEXING_TASK_PREFIX).ok_or_else(|| { anyhow!( "indexing task must contain the delimiter character `:`: `{}`", key @@ -177,7 +178,7 @@ fn parse_indexing_task_key(key: &str) -> anyhow::Result { /// ignored, just warnings are emitted. pub(crate) fn parse_indexing_tasks(node_state: &NodeState, node_id: &str) -> Vec { node_state - .key_values(|key, _| key.starts_with(INDEXING_TASK_PREFIX)) + .iter_prefix(INDEXING_TASK_PREFIX) .map(|(key, versioned_value)| { let indexing_task = parse_indexing_task_key(key)?; let num_tasks: usize = versioned_value.value.parse()?; diff --git a/quickwit/quickwit-cluster/src/node.rs b/quickwit/quickwit-cluster/src/node.rs index 8ca4d41abe4..1091ade7068 100644 --- a/quickwit/quickwit-cluster/src/node.rs +++ b/quickwit/quickwit-cluster/src/node.rs @@ -82,7 +82,7 @@ impl ClusterNode { for (indexing_task, indexing_tasks_group) in indexing_tasks.iter().group_by(|&task| task).into_iter() { - let key = format!("{INDEXING_TASK_PREFIX}:{}", indexing_task.to_string()); + let key = format!("{INDEXING_TASK_PREFIX}{}", indexing_task); node_state.set(key, indexing_tasks_group.count().to_string()); } Self::try_new(chitchat_id, &node_state, channel, is_self_node).unwrap() diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index eca434b5c43..70ab08492f8 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -21,7 +21,7 @@ use std::collections::hash_map::Entry; use std::num::NonZeroU32; use std::ops::RangeInclusive; use std::sync::Arc; -use std::time::Instant; +use std::time::{Duration, Instant}; use anyhow::Context; use async_trait::async_trait; @@ -38,7 +38,7 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_config::IndexingSettings; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::{IndexingPipelineId, PipelineMetrics}; use quickwit_proto::metastore::{ LastDeleteOpstampRequest, MetastoreService, MetastoreServiceClient, }; @@ -79,6 +79,10 @@ pub struct IndexerCounters { /// Number of (valid) documents in the current workbench. /// This value is used to trigger commit and for observation. pub num_docs_in_workbench: u64, + + /// Metrics describing the load and indexing performance of the + /// pipeline. This is only updated for cooperative indexers. + pub pipeline_metrics_opt: Option, } struct IndexerState { @@ -190,6 +194,7 @@ impl IndexerState { } else { None }; + let last_delete_opstamp_request = LastDeleteOpstampRequest { index_uid: self.pipeline_id.index_uid.to_string(), }; @@ -383,15 +388,25 @@ impl Actor for Indexer { let Some(indexing_workbench) = &self.indexing_workbench_opt else { return Ok(()); }; + let elapsed = indexing_workbench.create_instant.elapsed(); + let uncompressed_num_bytes = indexing_workbench + .indexed_splits + .values() + .map(|split| split.split_attrs.uncompressed_docs_size_in_bytes) + .sum::(); + self.update_pipeline_metrics(elapsed, uncompressed_num_bytes); + self.send_to_serializer(CommitTrigger::Drained, ctx).await?; let commit_timeout = self.indexer_state.indexing_settings.commit_timeout(); if elapsed >= commit_timeout { return Ok(()); } + // Time to take a nap. let sleep_for = commit_timeout - elapsed; + ctx.schedule_self_msg(sleep_for, Command::Resume).await; self.handle(Command::Pause, ctx).await?; Ok(()) @@ -530,6 +545,20 @@ impl Indexer { } } + fn update_pipeline_metrics(&mut self, elapsed: Duration, uncompressed_num_bytes: u64) { + let commit_timeout = self.indexer_state.indexing_settings.commit_timeout(); + let cpu_thousandth: u16 = if elapsed >= commit_timeout { + 1_000 + } else { + (elapsed.as_micros() * 1_000 / commit_timeout.as_micros()) as u16 + }; + self.counters.pipeline_metrics_opt = Some(PipelineMetrics { + cpu_thousandth, + throughput_mb_per_sec: (uncompressed_num_bytes / (elapsed.as_millis() as u64 * 1_000)) + as u16, + }); + } + fn memory_usage(&self) -> Byte { if let Some(workbench) = &self.indexing_workbench_opt { workbench.memory_usage @@ -797,6 +826,7 @@ mod tests { num_splits_emitted: 1, num_split_batches_emitted: 1, num_docs_in_workbench: 1, //< the num docs in split counter has been reset. + pipeline_metrics_opt: None, } ); let messages: Vec = index_serializer_inbox.drain_for_test_typed(); @@ -1039,6 +1069,7 @@ mod tests { num_splits_emitted: 1, num_split_batches_emitted: 1, num_docs_in_workbench: 0, + pipeline_metrics_opt: None, } ); let indexed_split_batches: Vec = @@ -1111,6 +1142,7 @@ mod tests { num_splits_emitted: 1, num_split_batches_emitted: 1, num_docs_in_workbench: 0, + pipeline_metrics_opt: None, } ); let output_messages: Vec = @@ -1201,6 +1233,7 @@ mod tests { num_docs_in_workbench: 2, num_splits_emitted: 0, num_split_batches_emitted: 0, + pipeline_metrics_opt: None, } ); universe.send_exit_with_success(&indexer_mailbox).await?; @@ -1212,6 +1245,7 @@ mod tests { num_docs_in_workbench: 0, num_splits_emitted: 2, num_split_batches_emitted: 1, + pipeline_metrics_opt: None, } ); let split_batches: Vec = @@ -1557,6 +1591,7 @@ mod tests { num_splits_emitted: 0, num_split_batches_emitted: 0, num_docs_in_workbench: 0, //< the num docs in split counter has been reset. + pipeline_metrics_opt: None, } ); diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 98bef9ac3f2..5e9b9864888 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -142,11 +142,11 @@ impl Actor for IndexingPipeline { async fn finalize( &mut self, _exit_status: &ActorExitStatus, - _ctx: &ActorContext, + ctx: &ActorContext, ) -> anyhow::Result<()> { // We update the observation to ensure our last "black box" observation // is up to date. - self.perform_observe().await; + self.perform_observe(ctx).await; Ok(()) } } @@ -237,7 +237,7 @@ impl IndexingPipeline { self.statistics.generation } - async fn perform_observe(&mut self) { + async fn perform_observe(&mut self, ctx: &ActorContext) { let Some(handles) = &self.handles_opt else { return; }; @@ -256,6 +256,9 @@ impl IndexingPipeline { ) .set_generation(self.statistics.generation) .set_num_spawn_attempts(self.statistics.num_spawn_attempts); + let pipeline_metrics_opt = handles.indexer.last_observation().pipeline_metrics_opt; + self.statistics.pipeline_metrics_opt = pipeline_metrics_opt; + ctx.observe(self); } /// Checks if some actors have terminated. @@ -490,7 +493,7 @@ impl Handler for IndexingPipeline { supervise_loop_token: SuperviseLoop, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - self.perform_observe().await; + self.perform_observe(ctx).await; self.perform_health_check(ctx).await?; ctx.schedule_self_msg(SUPERVISE_INTERVAL, supervise_loop_token) .await; @@ -575,6 +578,7 @@ pub struct IndexingPipelineParams { pub source_storage_resolver: StorageResolver, pub ingester_pool: IngesterPool, pub queues_dir_path: PathBuf, + pub event_broker: EventBroker, } diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 42c22d30f00..519a296e8c6 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -46,7 +46,7 @@ use quickwit_metastore::{ }; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, ApplyIndexingPlanResponse, IndexingError, IndexingPipelineId, - IndexingTask, + IndexingTask, PipelineMetrics, }; use quickwit_proto::metastore::{ IndexMetadataRequest, ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, @@ -303,6 +303,7 @@ impl IndexingService { let max_concurrent_split_uploads_index = (self.max_concurrent_split_uploads / 2).max(1); let max_concurrent_split_uploads_merge = (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); + let pipeline_params = IndexingPipelineParams { pipeline_id: pipeline_id.clone(), metastore: self.metastore.clone(), @@ -416,6 +417,19 @@ impl IndexingService { }); self.counters.num_running_merge_pipelines = self.merge_pipeline_handles.len(); self.update_cluster_running_indexing_tasks().await; + + let pipeline_metrics: HashMap<&IndexingPipelineId, PipelineMetrics> = self + .indexing_pipelines + .iter() + .flat_map(|(pipeline, (_, pipeline_handle))| { + let indexer_metrics = pipeline_handle.last_observation(); + let pipeline_metrics = indexer_metrics.pipeline_metrics_opt?; + Some((pipeline, pipeline_metrics)) + }) + .collect(); + self.cluster + .update_self_node_pipeline_metrics(&pipeline_metrics) + .await; Ok(()) } diff --git a/quickwit/quickwit-indexing/src/models/indexing_statistics.rs b/quickwit/quickwit-indexing/src/models/indexing_statistics.rs index 8d9c67acda7..bd404021d45 100644 --- a/quickwit/quickwit-indexing/src/models/indexing_statistics.rs +++ b/quickwit/quickwit-indexing/src/models/indexing_statistics.rs @@ -19,6 +19,7 @@ use std::sync::atomic::Ordering; +use quickwit_proto::indexing::PipelineMetrics; use serde::Serialize; use crate::actors::{DocProcessorCounters, IndexerCounters, PublisherCounters, UploaderCounters}; @@ -48,6 +49,8 @@ pub struct IndexingStatistics { pub generation: usize, /// Number of successive pipeline spawn attempts. pub num_spawn_attempts: usize, + // Pipeline metrics. + pub pipeline_metrics_opt: Option, } impl IndexingStatistics { diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index 2ed07cd10c2..9923edea31a 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -17,8 +17,9 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::fmt::Formatter; use std::hash::Hash; -use std::io; +use std::{fmt, io}; use anyhow::anyhow; use quickwit_actors::AskError; @@ -149,9 +150,15 @@ pub struct IndexingPipelineId { pub pipeline_ord: usize, } -impl ToString for IndexingTask { - fn to_string(&self) -> String { - format!("{}:{}", self.index_uid, self.source_id) +impl fmt::Display for IndexingPipelineId { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "{}:{}", self.index_uid, &self.source_id) + } +} + +impl fmt::Display for IndexingTask { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "{}:{}", self.index_uid, &self.source_id) } } @@ -200,6 +207,22 @@ impl TryFrom<&str> for IndexingTask { } } +#[derive(Clone, Copy, Debug, PartialEq, Eq, Serialize)] +pub struct PipelineMetrics { + pub cpu_thousandth: u16, + pub throughput_mb_per_sec: u16, +} + +impl fmt::Display for PipelineMetrics { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!( + f, + "{}‰,{}MB/s", + self.cpu_thousandth, self.throughput_mb_per_sec + ) + } +} + #[cfg(test)] mod tests { use super::*; From 8c1138fd4e6122c9c3ee1ee4f11c2952faf1bfb1 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Fri, 27 Oct 2023 14:55:29 -0400 Subject: [PATCH 03/12] Fix `IndexingStatistics`-related tests (#4040) --- quickwit/quickwit-indexing/src/actors/indexer.rs | 4 +++- quickwit/quickwit-indexing/src/lib.rs | 3 ++- quickwit/quickwit-proto/src/indexing/mod.rs | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index 70ab08492f8..cd19c98e938 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -1062,7 +1062,9 @@ mod tests { .await .unwrap(); universe.sleep(Duration::from_secs(3)).await; - let indexer_counters = indexer_handle.observe().await.state; + let mut indexer_counters = indexer_handle.observe().await.state; + indexer_counters.pipeline_metrics_opt = None; + assert_eq!( indexer_counters, IndexerCounters { diff --git a/quickwit/quickwit-indexing/src/lib.rs b/quickwit/quickwit-indexing/src/lib.rs index 3fd3932eea5..61c0bc655b5 100644 --- a/quickwit/quickwit-indexing/src/lib.rs +++ b/quickwit/quickwit-indexing/src/lib.rs @@ -24,6 +24,7 @@ use quickwit_cluster::Cluster; use quickwit_common::pubsub::EventBroker; use quickwit_config::NodeConfig; use quickwit_ingest::{IngestApiService, IngesterPool}; +use quickwit_proto::indexing::PipelineMetrics; use quickwit_proto::metastore::MetastoreServiceClient; use quickwit_storage::StorageResolver; use tracing::info; @@ -53,7 +54,7 @@ use self::merge_policy::MergePolicy; pub use self::source::check_source_connectivity; #[derive(utoipa::OpenApi)] -#[openapi(components(schemas(IndexingStatistics)))] +#[openapi(components(schemas(IndexingStatistics, PipelineMetrics)))] /// Schema used for the OpenAPI generation which are apart of this crate. pub struct IndexingApiSchemas; diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index 9923edea31a..e7c7b3273a3 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -207,7 +207,7 @@ impl TryFrom<&str> for IndexingTask { } } -#[derive(Clone, Copy, Debug, PartialEq, Eq, Serialize)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Serialize, utoipa::ToSchema)] pub struct PipelineMetrics { pub cpu_thousandth: u16, pub throughput_mb_per_sec: u16, From 572d6d50c196f7d48defaf624cb4bf568450ca68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Fran=C3=A7ois=20Massot?= Date: Fri, 27 Oct 2023 23:36:17 +0200 Subject: [PATCH 04/12] Fix ui index view. (#4041) --- quickwit/quickwit-ui/cypress/e2e/homepage.spec.cy.js | 6 ++++++ quickwit/quickwit-ui/src/services/client.ts | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/quickwit/quickwit-ui/cypress/e2e/homepage.spec.cy.js b/quickwit/quickwit-ui/cypress/e2e/homepage.spec.cy.js index 15779277436..5067b5583c4 100644 --- a/quickwit/quickwit-ui/cypress/e2e/homepage.spec.cy.js +++ b/quickwit/quickwit-ui/cypress/e2e/homepage.spec.cy.js @@ -32,4 +32,10 @@ describe('Home navigation', () => { cy.get('p').should('contain.text', 'Cluster'); cy.get('span').should('contain.text', 'cluster_id'); }); + it('Should display otel logs index page', () => { + cy.visit('http://127.0.0.1:7280/ui/indexes/otel-logs-v0_6'); + cy.get('a') + .should('be.visible') + .should('contain.text', 'Indexes') + }); }) diff --git a/quickwit/quickwit-ui/src/services/client.ts b/quickwit/quickwit-ui/src/services/client.ts index 6acb2c056a5..9f55d41ba8f 100644 --- a/quickwit/quickwit-ui/src/services/client.ts +++ b/quickwit/quickwit-ui/src/services/client.ts @@ -75,9 +75,10 @@ export class Client { } async getAllSplits(indexId: string): Promise> { - const splits: Array = await this.fetch(`${this.apiRoot()}indexes/${indexId}/splits`, {}); + // TODO: restrieve all the splits. + const results: {splits: Array} = await this.fetch(`${this.apiRoot()}indexes/${indexId}/splits?limit=10000`, {}); - return splits; + return results['splits']; } async listIndexes(): Promise> { From 742347b4dfefb42ca3f5619112ff5f689980ee08 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Fri, 27 Oct 2023 19:48:13 -0400 Subject: [PATCH 05/12] Add retry logic for router (#4027) * Add retry logic for router * Fix various issues --- quickwit/Cargo.lock | 91 +++--- .../src/control_plane.rs | 33 +- .../src/control_plane_model.rs | 82 +++-- .../src/indexing_plan.rs | 6 +- .../src/ingest/ingest_controller.rs | 78 +++-- .../src/source/ingest/mod.rs | 148 +++++++-- quickwit/quickwit-ingest/Cargo.toml | 1 + .../quickwit-ingest/src/ingest_v2/fetch.rs | 45 ++- .../quickwit-ingest/src/ingest_v2/ingest.md | 2 +- .../quickwit-ingest/src/ingest_v2/ingester.rs | 15 +- quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 3 +- .../src/ingest_v2/replication.rs | 9 + .../quickwit-ingest/src/ingest_v2/router.rs | 298 +++++++++++++++--- .../src/ingest_v2/workbench.rs | 279 ++++++++++++++++ .../file_backed_index/shards.rs | 56 ++-- .../metastore/file_backed_metastore/mod.rs | 1 + .../quickwit-metastore/src/tests/shard.rs | 8 +- .../protos/quickwit/ingest.proto | 14 +- .../protos/quickwit/ingester.proto | 63 ++-- .../protos/quickwit/metastore.proto | 25 +- .../protos/quickwit/router.proto | 40 ++- .../quickwit/quickwit.ingest.ingester.rs | 83 +++-- .../quickwit/quickwit.ingest.router.rs | 75 +++-- .../src/codegen/quickwit/quickwit.ingest.rs | 42 +-- .../codegen/quickwit/quickwit.metastore.rs | 71 +++-- .../quickwit-proto/src/control_plane/mod.rs | 11 +- quickwit/quickwit-proto/src/metastore/mod.rs | 13 +- quickwit/quickwit-proto/src/types/mod.rs | 2 + .../src/ingest_api/rest_handler.rs | 1 + 29 files changed, 1177 insertions(+), 418 deletions(-) create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/workbench.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index de278b96d92..9a7ae05bb2b 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -1447,9 +1447,9 @@ checksum = "e496a50fda8aacccc86d7529e2c1e0892dbd0f898a6b5645b5561b89c3210efa" [[package]] name = "cpufeatures" -version = "0.2.10" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fbc60abd742b35f2492f808e1abbb83d45f72db402e14c55057edc9c7b1e9e4" +checksum = "ce420fe07aecd3e67c5f910618fe65e94158f6dcc0adf44e00d69ce2bdfe0fd0" dependencies = [ "libc", ] @@ -2256,7 +2256,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "29f9df8a11882c4e3335eb2d18a0137c505d9ca927470b0cac9c6f0ae07d28f7" dependencies = [ - "rustix 0.38.20", + "rustix 0.38.21", "windows-sys 0.48.0", ] @@ -2268,9 +2268,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23342abe12aba583913b2e62f22225ff9c950774065e4bfb61a19cd9770fec40" +checksum = "da0290714b38af9b4a7b094b8a37086d1b4e61f2df9122c3cad2577669145335" dependencies = [ "futures-channel", "futures-core", @@ -2283,9 +2283,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "955518d47e09b25bbebc7a18df10b81f0c766eaf4c4f1cccef2fca5f2a4fb5f2" +checksum = "ff4dd66668b557604244583e3e1e1eada8c5c2e96a6d0d6653ede395b78bbacb" dependencies = [ "futures-core", "futures-sink", @@ -2293,15 +2293,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bca583b7e26f571124fe5b7561d49cb2868d79116cfa0eefce955557c6fee8c" +checksum = "eb1d22c66e66d9d72e1758f0bd7d4fd0bee04cad842ee34587d68c07e45d088c" [[package]] name = "futures-executor" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccecee823288125bd88b4d7f565c9e58e41858e47ab72e8ea2d64e93624386e0" +checksum = "0f4fb8693db0cf099eadcca0efe2a5a22e4550f98ed16aba6c48700da29597bc" dependencies = [ "futures-core", "futures-task", @@ -2321,9 +2321,9 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fff74096e71ed47f8e023204cfd0aa1289cd54ae5430a9523be060cdb849964" +checksum = "8bf34a163b5c4c52d0478a4d757da8fb65cabef42ba90515efee0f6f9fa45aaa" [[package]] name = "futures-lite" @@ -2342,9 +2342,9 @@ dependencies = [ [[package]] name = "futures-macro" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" +checksum = "53b153fd91e4b0147f4aced87be237c98248656bb01050b96bf3ee89220a8ddb" dependencies = [ "proc-macro2", "quote", @@ -2353,15 +2353,15 @@ dependencies = [ [[package]] name = "futures-sink" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f43be4fe21a13b9781a69afa4985b0f6ee0e1afab2c6f454a8cf30e2b2237b6e" +checksum = "e36d3378ee38c2a36ad710c5d30c2911d752cb941c00c72dbabfb786a7970817" [[package]] name = "futures-task" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76d3d132be6c0e6aa1534069c705a74a5997a356c0dc2f86a47765e5617c5b65" +checksum = "efd193069b0ddadc69c46389b740bbccdd97203899b48d09c5f7969591d6bae2" [[package]] name = "futures-timer" @@ -2371,9 +2371,9 @@ checksum = "e64b03909df88034c26dc1547e8970b91f98bdb65165d6a4e9110d94263dbb2c" [[package]] name = "futures-util" -version = "0.3.28" +version = "0.3.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26b01e40b772d54cf6c6d721c1d1abd0647a0106a12ecaa1c186273392a69533" +checksum = "a19526d624e703a3179b3d322efec918b6246ea0fa51d41124525f00f1cc8104" dependencies = [ "futures-channel", "futures-core", @@ -2895,9 +2895,9 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.24.1" +version = "0.24.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d78e1e73ec14cf7375674f74d7dde185c8206fd9dea6fb6295e8a98098aaa97" +checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" dependencies = [ "futures-util", "http", @@ -3081,7 +3081,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cb0889898416213fab133e1d33a0e5858a48177452750691bde3666d0fdbaf8b" dependencies = [ "hermit-abi", - "rustix 0.38.20", + "rustix 0.38.21", "windows-sys 0.48.0", ] @@ -4044,9 +4044,9 @@ checksum = "ff011a302c396a5197692431fc1948019154afc178baf7d8e37367442a4601cf" [[package]] name = "openssl-src" -version = "300.1.5+3.1.3" +version = "300.1.6+3.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "559068e4c12950d7dcaa1857a61725c0d38d4fc03ff8e070ab31a75d6e316491" +checksum = "439fac53e092cd7442a3660c85dde4643ab3b5bd39040912388dcdabf6b88085" dependencies = [ "cc", ] @@ -4732,7 +4732,7 @@ dependencies = [ "byteorder", "hex", "lazy_static", - "rustix 0.36.16", + "rustix 0.36.17", ] [[package]] @@ -4939,7 +4939,7 @@ dependencies = [ "aws-types", "futures", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls 0.24.2", "once_cell", "quickwit-common", "rand 0.8.5", @@ -5359,6 +5359,7 @@ dependencies = [ "futures", "http", "hyper", + "itertools 0.11.0", "mockall", "mrecordlog", "once_cell", @@ -6104,7 +6105,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls 0.24.1", + "hyper-rustls 0.24.2", "hyper-tls", "ipnet", "js-sys", @@ -6217,16 +6218,14 @@ dependencies = [ [[package]] name = "rsa" -version = "0.9.2" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ab43bb47d23c1a631b4b680199a45255dce26fa9ab2fa902581f624ff13e6a8" +checksum = "86ef35bf3e7fe15a53c4ab08a998e42271eab13eb0db224126bc7bc4c4bad96d" dependencies = [ - "byteorder", "const-oid", "digest", "num-bigint-dig", "num-integer", - "num-iter", "num-traits", "pkcs1", "pkcs8", @@ -6320,9 +6319,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.36.16" +version = "0.36.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6da3636faa25820d8648e0e31c5d519bbb01f72fdf57131f0f5f7da5fed36eab" +checksum = "305efbd14fde4139eb501df5f136994bb520b033fa9fbdce287507dc23b8c7ed" dependencies = [ "bitflags 1.3.2", "errno", @@ -6334,9 +6333,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.20" +version = "0.38.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67ce50cb2e16c2903e30d1cbccfd8387a74b9d4c938b6a4c5ec6cc7556f7a8a0" +checksum = "2b426b0506e5d50a7d8dafcf2e81471400deb602392c7dd110815afb4eaf02a3" dependencies = [ "bitflags 2.4.1", "errno", @@ -7420,14 +7419,14 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.8.0" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb94d2f3cc536af71caac6b6fcebf65860b347e7ce0cc9ebe8f70d3e521054ef" +checksum = "7ef1adac450ad7f4b3c28589471ade84f25f731a7a0fe30d71dfa9f60fd808e5" dependencies = [ "cfg-if", "fastrand 2.0.1", - "redox_syscall 0.3.5", - "rustix 0.38.20", + "redox_syscall 0.4.1", + "rustix 0.38.21", "windows-sys 0.48.0", ] @@ -8643,7 +8642,7 @@ dependencies = [ "either", "home", "once_cell", - "rustix 0.38.20", + "rustix 0.38.21", ] [[package]] @@ -8948,18 +8947,18 @@ checksum = "09041cd90cf85f7f8b2df60c646f853b7f535ce68f85244eb6731cf89fa498ec" [[package]] name = "zerocopy" -version = "0.7.14" +version = "0.7.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69c48d63854f77746c68a5fbb4aa17f3997ece1cb301689a257af8cb80610d21" +checksum = "c552e97c5a9b90bc8ddc545b5106e798807376356688ebaa3aee36f44f8c4b9e" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.14" +version = "0.7.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c258c1040279e4f88763a113de72ce32dde2d50e2a94573f15dd534cea36a16d" +checksum = "964bc0588d7ac1c0243d0427ef08482618313702bbb014806cb7ab3da34d3d99" dependencies = [ "proc-macro2", "quote", diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 39e71b06419..c6b3e80e089 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -180,11 +180,12 @@ fn convert_metastore_error( | MetastoreError::JsonDeserializeError { .. } | MetastoreError::JsonSerializeError { .. } | MetastoreError::NotFound(_) => true, - MetastoreError::Unavailable(_) + MetastoreError::Connection { .. } + | MetastoreError::Db { .. } + | MetastoreError::InconsistentControlPlaneState { .. } | MetastoreError::Internal { .. } | MetastoreError::Io { .. } - | MetastoreError::Connection { .. } - | MetastoreError::Db { .. } => false, + | MetastoreError::Unavailable(_) => false, }; if is_transaction_certainly_aborted { // If the metastore transaction is certain to have been aborted, @@ -363,10 +364,22 @@ impl Handler for ControlPlane { request: GetOrCreateOpenShardsRequest, ctx: &ActorContext, ) -> Result { - Ok(self + let response = match self .ingest_controller .get_or_create_open_shards(request, &mut self.model, ctx.progress()) - .await) + .await + { + Ok(response) => response, + Err(ControlPlaneError::Metastore(metastore_error)) => { + return convert_metastore_error(metastore_error); + } + Err(control_plane_error) => { + return Ok(Err(control_plane_error)); + } + }; + // TODO: Why do we return an error if the indexing scheduler fails? + self.indexing_scheduler.on_index_change(&self.model).await?; + Ok(Ok(response)) } } @@ -379,7 +392,7 @@ mod tests { ListIndexesMetadataResponseExt, }; use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; - use quickwit_proto::ingest::Shard; + use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ EntityKind, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListShardsSubresponse, MetastoreError, SourceType, @@ -641,7 +654,7 @@ mod tests { } #[tokio::test] - async fn test_control_plane_get_open_shards() { + async fn test_control_plane_get_or_create_open_shards() { let universe = Universe::with_accelerated_time(); let cluster_id = "test-cluster".to_string(); @@ -675,6 +688,7 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: INGEST_SOURCE_ID.to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, ..Default::default() }], next_shard_id: 2, @@ -716,11 +730,6 @@ mod tests { universe.assert_quit().await; } - #[tokio::test] - async fn test_control_plane_close_shards() { - // TODO: Write test when the RPC is actually called by ingesters. - } - #[tokio::test] async fn test_control_plane_supervision_reload_from_metastore() { let universe = Universe::default(); diff --git a/quickwit/quickwit-control-plane/src/control_plane_model.rs b/quickwit/quickwit-control-plane/src/control_plane_model.rs index 98952293380..8fdb075ee9e 100644 --- a/quickwit/quickwit-control-plane/src/control_plane_model.rs +++ b/quickwit/quickwit-control-plane/src/control_plane_model.rs @@ -291,16 +291,16 @@ impl ControlPlaneModel { self.index_uid_table.get(index_id).cloned() } - /// Updates the shard table. - pub fn update_shards( + /// Inserts the shards that have just been opened by calling `open_shards` on the metastore. + pub fn insert_newly_opened_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, - shards: &[Shard], + shards: Vec, next_shard_id: NextShardId, ) { self.shard_table - .update_shards(index_uid, source_id, shards, next_shard_id); + .insert_newly_opened_shards(index_uid, source_id, shards, next_shard_id); } /// Finds open shards for a given index and source and whose leaders are not in the set of @@ -309,10 +309,10 @@ impl ControlPlaneModel { &self, index_uid: &IndexUid, source_id: &SourceId, - unavailable_ingesters: &FnvHashSet, + unavailable_leaders: &FnvHashSet, ) -> Option<(Vec, NextShardId)> { self.shard_table - .find_open_shards(index_uid, source_id, unavailable_ingesters) + .find_open_shards(index_uid, source_id, unavailable_leaders) } } @@ -375,7 +375,7 @@ impl ShardTable { &self, index_uid: &IndexUid, source_id: &SourceId, - unavailable_ingesters: &FnvHashSet, + unavailable_leaders: &FnvHashSet, ) -> Option<(Vec, NextShardId)> { let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -387,7 +387,7 @@ impl ShardTable { .values() .filter(|shard| { shard.is_open() - && !unavailable_ingesters.contains(NodeIdRef::from_str(&shard.leader_id)) + && !unavailable_leaders.contains(NodeIdRef::from_str(&shard.leader_id)) }) .cloned() .collect(); @@ -402,11 +402,11 @@ impl ShardTable { } /// Updates the shard table. - pub fn update_shards( + pub fn insert_newly_opened_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, - shards: &[Shard], + opened_shards: Vec, next_shard_id: NextShardId, ) { let source_uid = SourceUid { @@ -415,19 +415,24 @@ impl ShardTable { }; match self.table_entries.entry(source_uid) { Entry::Occupied(mut entry) => { - for shard in shards { - let table_entry = entry.get_mut(); - table_entry.shards.insert(shard.shard_id, shard.clone()); - table_entry.next_shard_id = next_shard_id; + let table_entry = entry.get_mut(); + + for opened_shard in opened_shards { + // We only insert shards that we don't know about because the control plane + // knows more about the state of the shards than the metastore. + table_entry + .shards + .entry(opened_shard.shard_id) + .or_insert(opened_shard); } + table_entry.next_shard_id = next_shard_id; } // This should never happen if the control plane view is consistent with the state of // the metastore, so should we panic here? Warnings are most likely going to go // unnoticed. Entry::Vacant(entry) => { - let shards: FnvHashMap = shards - .iter() - .cloned() + let shards: FnvHashMap = opened_shards + .into_iter() .map(|shard| (shard.shard_id, shard)) .collect(); let table_entry = ShardTableEntry { @@ -558,10 +563,10 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.update_shards( + shard_table.insert_newly_opened_shards( &index_uid, &source_id, - &[shard_01, shard_02, shard_03.clone(), shard_04.clone()], + vec![shard_01, shard_02, shard_03.clone(), shard_04.clone()], 5, ); let (open_shards, next_shard_id) = shard_table @@ -583,13 +588,13 @@ mod tests { } #[test] - fn test_shard_table_update_shards() { + fn test_shard_table_insert_newly_opened_shards() { let index_uid_0: IndexUid = "test-index:0".into(); let source_id = "test-source".to_string(); let mut shard_table = ShardTable::default(); - let mut shard_01 = Shard { + let shard_01 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), shard_id: 1, @@ -597,7 +602,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.update_shards(&index_uid_0, &source_id, &[shard_01.clone()], 2); + shard_table.insert_newly_opened_shards(&index_uid_0, &source_id, vec![shard_01.clone()], 2); assert_eq!(shard_table.table_entries.len(), 1); @@ -611,7 +616,14 @@ mod tests { assert_eq!(shards[0], shard_01); assert_eq!(table_entry.next_shard_id, 2); - shard_01.shard_state = ShardState::Closed as i32; + shard_table + .table_entries + .get_mut(&source_uid) + .unwrap() + .shards + .get_mut(&1) + .unwrap() + .shard_state = ShardState::Unavailable as i32; let shard_02 = Shard { index_uid: index_uid_0.clone().into(), @@ -622,10 +634,10 @@ mod tests { ..Default::default() }; - shard_table.update_shards( + shard_table.insert_newly_opened_shards( &index_uid_0, &source_id, - &[shard_01.clone(), shard_02.clone()], + vec![shard_01.clone(), shard_02.clone()], 3, ); @@ -638,7 +650,7 @@ mod tests { let table_entry = shard_table.table_entries.get(&source_uid).unwrap(); let shards = table_entry.shards(); assert_eq!(shards.len(), 2); - assert_eq!(shards[0], shard_01); + assert_eq!(shards[0].shard_state(), ShardState::Unavailable); assert_eq!(shards[1], shard_02); assert_eq!(table_entry.next_shard_id, 3); } @@ -675,8 +687,13 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.update_shards(&index_uid_0, &source_id, &[shard_01, shard_02], 3); - shard_table.update_shards(&index_uid_0, &source_id, &[shard_11], 2); + shard_table.insert_newly_opened_shards( + &index_uid_0, + &source_id, + vec![shard_01, shard_02], + 3, + ); + shard_table.insert_newly_opened_shards(&index_uid_0, &source_id, vec![shard_11], 2); let closed_shard_ids = shard_table.close_shards(&index_uid_0, &source_id, &[1, 2, 3]); assert_eq!(closed_shard_ids, &[1]); @@ -722,8 +739,13 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.update_shards(&index_uid_0, &source_id, &[shard_01.clone(), shard_02], 3); - shard_table.update_shards(&index_uid_1, &source_id, &[shard_11], 2); + shard_table.insert_newly_opened_shards( + &index_uid_0, + &source_id, + vec![shard_01.clone(), shard_02], + 3, + ); + shard_table.insert_newly_opened_shards(&index_uid_1, &source_id, vec![shard_11], 2); shard_table.delete_shards(&index_uid_0, &source_id, &[2]); shard_table.delete_shards(&index_uid_1, &source_id, &[1]); diff --git a/quickwit/quickwit-control-plane/src/indexing_plan.rs b/quickwit/quickwit-control-plane/src/indexing_plan.rs index 045900655b5..c3f1d14e8e5 100644 --- a/quickwit/quickwit-control-plane/src/indexing_plan.rs +++ b/quickwit/quickwit-control-plane/src/indexing_plan.rs @@ -287,7 +287,11 @@ pub(crate) fn list_indexing_tasks( continue; } match source_config.source_type() { - SourceType::Cli | SourceType::File | SourceType::Vec | SourceType::Void => { + SourceType::Cli + | SourceType::File + | SourceType::Unspecified + | SourceType::Vec + | SourceType::Void => { continue; } SourceType::IngestV1 => { diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index b2a51957427..a5a3c17a1ed 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -302,22 +302,26 @@ impl IngestController { let open_shards_response = progress .protect_future(self.metastore.open_shards(open_shards_request)) .await?; - for open_shards_subresponse in &open_shards_response.subresponses { + for open_shards_subresponse in open_shards_response.subresponses { let index_uid: IndexUid = open_shards_subresponse.index_uid.clone().into(); - model.update_shards( + let source_id = open_shards_subresponse.source_id.clone(); + + model.insert_newly_opened_shards( &index_uid, - &open_shards_subresponse.source_id, - &open_shards_subresponse.open_shards, + &source_id, + open_shards_subresponse.opened_shards, open_shards_subresponse.next_shard_id, ); - } - for open_shards_subresponse in open_shards_response.subresponses { - let get_open_shards_subresponse = GetOpenShardsSubresponse { - index_uid: open_shards_subresponse.index_uid, - source_id: open_shards_subresponse.source_id, - open_shards: open_shards_subresponse.open_shards, - }; - get_open_shards_subresponses.push(get_open_shards_subresponse); + if let Some((open_shards, _next_shard_id)) = + model.find_open_shards(&index_uid, &source_id, &unavailable_leaders) + { + let get_open_shards_subresponse = GetOpenShardsSubresponse { + index_uid: index_uid.into(), + source_id, + open_shards, + }; + get_open_shards_subresponses.push(get_open_shards_subresponse); + } } } Ok(GetOrCreateOpenShardsResponse { @@ -548,35 +552,36 @@ mod tests { } #[tokio::test] - async fn test_ingest_controller_get_open_shards() { + async fn test_ingest_controller_get_or_create_open_shards() { let source_id: &'static str = "test-source"; - let index_0 = "test-index-0"; - let index_metadata_0 = IndexMetadata::for_test(index_0, "ram://indexes/test-index0"); + let index_id_0 = "test-index-0"; + let index_metadata_0 = IndexMetadata::for_test(index_id_0, "ram://indexes/test-index-0"); let index_uid_0 = index_metadata_0.index_uid.clone(); - let index_uid_0_str = index_uid_0.to_string(); - let index_1 = "test-index-1"; - let index_metadata_1 = IndexMetadata::for_test(index_1, "ram://indexes/test-index1"); + + let index_id_1 = "test-index-1"; + let index_metadata_1 = IndexMetadata::for_test(index_id_1, "ram://indexes/test-index-1"); let index_uid_1 = index_metadata_1.index_uid.clone(); - let index_uid_1_str = index_uid_1.to_string(); let progress = Progress::default(); - let index_uid_1_str_clone = index_uid_1_str.clone(); let mut mock_metastore = MetastoreServiceClient::mock(); - mock_metastore - .expect_open_shards() - .once() - .returning(move |request| { + mock_metastore.expect_open_shards().once().returning({ + let index_uid_1 = index_uid_1.clone(); + + move |request| { assert_eq!(request.subrequests.len(), 1); - assert_eq!(&request.subrequests[0].index_uid, &index_uid_1_str_clone); + assert_eq!(&request.subrequests[0].index_uid, index_uid_1.as_str()); assert_eq!(&request.subrequests[0].source_id, source_id); let subresponses = vec![metastore::OpenShardsSubresponse { - index_uid: index_uid_1_str_clone.to_string(), - source_id: "test-source".to_string(), - open_shards: vec![Shard { + index_uid: index_uid_1.clone().into(), + source_id: source_id.to_string(), + opened_shards: vec![Shard { + index_uid: index_uid_1.clone().into(), + source_id: source_id.to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, leader_id: "test-ingester-2".to_string(), ..Default::default() }], @@ -584,7 +589,8 @@ mod tests { }]; let response = metastore::OpenShardsResponse { subresponses }; Ok(response) - }); + } + }); let ingester_pool = IngesterPool::default(); let mut mock_ingester = MockIngesterService::default(); @@ -621,12 +627,16 @@ mod tests { let shards = vec![ Shard { + index_uid: index_uid_0.clone().into(), + source_id: source_id.to_string(), shard_id: 1, leader_id: "test-ingester-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }, Shard { + index_uid: index_uid_0.clone().into(), + source_id: source_id.to_string(), shard_id: 2, leader_id: "test-ingester-1".to_string(), shard_state: ShardState::Open as i32, @@ -634,7 +644,7 @@ mod tests { }, ]; - model.update_shards(&index_uid_0, &source_id.into(), &shards, 3); + model.insert_newly_opened_shards(&index_uid_0, &source_id.into(), shards, 3); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), @@ -673,7 +683,7 @@ mod tests { assert_eq!(response.subresponses.len(), 2); - assert_eq!(response.subresponses[0].index_uid, index_uid_0_str); + assert_eq!(response.subresponses[0].index_uid, index_uid_0.as_str()); assert_eq!(response.subresponses[0].source_id, source_id); assert_eq!(response.subresponses[0].open_shards.len(), 1); assert_eq!(response.subresponses[0].open_shards[0].shard_id, 2); @@ -682,7 +692,7 @@ mod tests { "test-ingester-1" ); - assert_eq!(&response.subresponses[1].index_uid, &index_uid_1_str); + assert_eq!(&response.subresponses[1].index_uid, index_uid_1.as_str()); assert_eq!(response.subresponses[1].source_id, source_id); assert_eq!(response.subresponses[1].open_shards.len(), 1); assert_eq!(response.subresponses[1].open_shards[0].shard_id, 1); @@ -713,7 +723,7 @@ mod tests { shard_state: ShardState::Open as i32, ..Default::default() }]; - model.update_shards(&index_uid, &source_id, &shards, 3); + model.insert_newly_opened_shards(&index_uid, &source_id, shards, 3); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), @@ -772,7 +782,7 @@ mod tests { ..Default::default() }, ]; - model.update_shards(&index_uid, &source_id, &shards, 4); + model.insert_newly_opened_shards(&index_uid, &source_id, shards, 4); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index e218331580f..1d7aee86ed1 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -26,13 +26,16 @@ use anyhow::{bail, Context}; use async_trait::async_trait; use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox}; -use quickwit_ingest::{decoded_mrecords, IngesterPool, MRecord, MultiFetchStream}; +use quickwit_ingest::{ + decoded_mrecords, FetchStreamError, IngesterPool, MRecord, MultiFetchStream, +}; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::ingest::ingester::{ FetchResponseV2, IngesterService, TruncateRequest, TruncateSubrequest, }; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsSubrequest, MetastoreService, MetastoreServiceClient, + AcquireShardsRequest, AcquireShardsSubrequest, AcquireShardsSubresponse, MetastoreService, + MetastoreServiceClient, }; use quickwit_proto::types::{IndexUid, NodeId, Position, PublishToken, ShardId, SourceId}; use serde_json::json; @@ -75,6 +78,16 @@ struct ClientId { pipeline_ord: usize, } +impl fmt::Display for ClientId { + fn fmt(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + write!( + formatter, + "indexer/{}/{}/{}/{}", + self.node_id, self.index_uid, self.source_id, self.pipeline_ord + ) + } +} + impl ClientId { fn new(node_id: NodeId, index_uid: IndexUid, source_id: SourceId, pipeline_ord: usize) -> Self { Self { @@ -85,20 +98,32 @@ impl ClientId { } } - fn client_id(&self) -> String { - format!( - "indexer/{}/{}/{}/{}", - self.node_id, self.index_uid, self.source_id, self.pipeline_ord - ) + #[cfg(not(test))] + fn new_publish_token(&self) -> String { + format!("{}/{}", self, Ulid::new()) + } + + #[cfg(test)] + fn new_publish_token(&self) -> String { + format!("{}/{}", self, Ulid::nil()) } } +#[derive(Debug, Clone, Copy, Default, Eq, PartialEq)] +enum IndexingStatus { + #[default] + Active, + Complete, + Error, +} + #[derive(Debug, Eq, PartialEq)] struct AssignedShard { leader_id: NodeId, follower_id_opt: Option, partition_id: PartitionId, current_position_inclusive: Position, + status: IndexingStatus, } /// Streams documents from a set of shards. @@ -123,9 +148,9 @@ impl IngestSource { runtime_args: Arc, _checkpoint: SourceCheckpoint, ) -> anyhow::Result { - let node_id: NodeId = runtime_args.node_id().into(); + let self_node_id: NodeId = runtime_args.node_id().into(); let client_id = ClientId::new( - node_id.clone(), + self_node_id.clone(), runtime_args.index_uid().clone(), runtime_args.source_id().to_string(), runtime_args.pipeline_ord(), @@ -134,9 +159,9 @@ impl IngestSource { let ingester_pool = runtime_args.ingester_pool.clone(); let assigned_shards = HashMap::new(); let fetch_stream = - MultiFetchStream::new(node_id, client_id.client_id(), ingester_pool.clone()); + MultiFetchStream::new(self_node_id, client_id.to_string(), ingester_pool.clone()); let publish_lock = PublishLock::default(); - let publish_token = format!("{}:{}", client_id.client_id(), Ulid::new()); + let publish_token = client_id.new_publish_token(); Ok(Self { client_id, @@ -175,6 +200,7 @@ impl IngestSource { batch_builder.force_commit(); } MRecord::Eof => { + assigned_shard.status = IndexingStatus::Complete; break; } MRecord::Unknown => { @@ -194,6 +220,23 @@ impl IngestSource { Ok(()) } + fn process_fetch_stream_error(&mut self, fetch_stream_error: FetchStreamError) { + if let Some(shard) = self.assigned_shards.get_mut(&fetch_stream_error.shard_id) { + if shard.status != IndexingStatus::Complete { + shard.status = IndexingStatus::Error; + } + } + } + + fn contains_publish_token(&self, subresponse: &AcquireShardsSubresponse) -> bool { + if let Some(acquired_shard) = subresponse.acquired_shards.get(0) { + if let Some(publish_token) = &acquired_shard.publish_token { + return *publish_token == self.publish_token; + } + } + false + } + async fn truncate(&self, truncation_point: &[(ShardId, Position)]) { let mut per_ingester_truncate_subrequests: HashMap<&NodeId, Vec> = HashMap::new(); @@ -270,10 +313,11 @@ impl Source for IngestSource { break; } } - Ok(Err(error)) => { - error!(error=?error, "failed to fetch payload"); + Ok(Err(fetch_stream_error)) => { + self.process_fetch_stream_error(fetch_stream_error); } Err(_) => { + // The deadline has elapsed. break; } } @@ -314,16 +358,18 @@ impl Source for IngestSource { } info!("new shard assignment: `{:?}`", new_assigned_shard_ids); + self.assigned_shards.clear(); + self.fetch_stream.reset(); self.publish_lock.kill().await; - self.publish_lock = PublishLock::default(); + self.publish_token = self.client_id.new_publish_token(); + ctx.send_message( doc_processor_mailbox, NewPublishLock(self.publish_lock.clone()), ) .await?; - self.publish_token = format!("{}:{}", self.client_id.client_id(), Ulid::new()); ctx.send_message( doc_processor_mailbox, NewPublishToken(self.publish_token.clone()), @@ -344,18 +390,11 @@ impl Source for IngestSource { .await .context("failed to acquire shards")?; - let Some(acquire_shards_subresponse) = acquire_shards_response + let acquire_shards_subresponse = acquire_shards_response .subresponses .into_iter() - .find(|subresponse| { - self.client_id.index_uid.as_str() == subresponse.index_uid - && subresponse.source_id == self.client_id.source_id - }) - else { - return Ok(()); - }; - self.assigned_shards.clear(); - self.fetch_stream.reset(); + .find(|subresponse| self.contains_publish_token(subresponse)) + .context("acquire shards response is empty")?; let mut truncation_point = Vec::with_capacity(acquire_shards_subresponse.acquired_shards.len()); @@ -373,7 +412,9 @@ impl Source for IngestSource { let from_position_exclusive = current_position_inclusive.clone(); let to_position_inclusive = Position::Eof; - if let Err(error) = ctx + let status = if from_position_exclusive == Position::Eof { + IndexingStatus::Complete + } else if let Err(error) = ctx .protect_future(self.fetch_stream.subscribe( leader_id.clone(), follower_id_opt.clone(), @@ -386,8 +427,10 @@ impl Source for IngestSource { .await { error!(error=%error, "failed to subscribe to shard"); - continue; - } + IndexingStatus::Error + } else { + IndexingStatus::Active + }; truncation_point.push((shard_id, current_position_inclusive.clone())); let assigned_shard = AssignedShard { @@ -395,6 +438,7 @@ impl Source for IngestSource { follower_id_opt, partition_id, current_position_inclusive, + status, }; self.assigned_shards.insert(shard_id, assigned_shard); } @@ -423,7 +467,7 @@ impl Source for IngestSource { fn observable_state(&self) -> serde_json::Value { json!({ - "client_id": self.client_id.client_id(), + "client_id": self.client_id.to_string(), "assigned_shards": self.assigned_shards.keys().copied().collect::>(), "publish_token": self.publish_token, }) @@ -440,7 +484,7 @@ mod tests { use quickwit_config::{SourceConfig, SourceParams}; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::ingest::ingester::{IngesterServiceClient, TruncateResponse}; - use quickwit_proto::ingest::{MRecordBatch, Shard}; + use quickwit_proto::ingest::{IngestV2Error, MRecordBatch, Shard, ShardState}; use quickwit_proto::metastore::{AcquireShardsResponse, AcquireShardsSubresponse}; use quickwit_storage::StorageResolver; use tokio::sync::watch; @@ -458,6 +502,9 @@ mod tests { pipeline_ord: 0, }; let source_config = SourceConfig::for_test("test-source", SourceParams::Ingest); + let publish_token = + "indexer/test-node/test-index:0/test-source/0/00000000000000000000000000"; + let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore .expect_acquire_shards() @@ -480,8 +527,9 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, publish_position_inclusive: Some(11u64.into()), - ..Default::default() + publish_token: Some(publish_token.to_string()), }], }], }; @@ -552,7 +600,7 @@ mod tests { shard_ids: vec![1, 2], }; let publish_lock = source.publish_lock.clone(); - let publish_token = source.publish_token.clone(); + // let publish_token = source.publish_token.clone(); source .assign_shards(assignment, &doc_processor_mailbox, &ctx) @@ -568,7 +616,7 @@ mod tests { .unwrap(); assert_eq!(&source.publish_lock, &publish_lock); - assert!(publish_token != source.publish_token); + // assert!(publish_token != source.publish_token); let NewPublishToken(publish_token) = doc_processor_inbox .recv_typed_message::() @@ -584,6 +632,7 @@ mod tests { follower_id_opt: None, partition_id: 1u64.into(), current_position_inclusive: 11u64.into(), + status: IndexingStatus::Active, }; assert_eq!(assigned_shard, &expected_assigned_shard); @@ -632,6 +681,7 @@ mod tests { follower_id_opt: None, partition_id: 1u64.into(), current_position_inclusive: 11u64.into(), + status: IndexingStatus::Active, }, ); source.assigned_shards.insert( @@ -641,6 +691,7 @@ mod tests { follower_id_opt: None, partition_id: 2u64.into(), current_position_inclusive: 22u64.into(), + status: IndexingStatus::Active, }, ); let fetch_response_tx = source.fetch_stream.fetch_response_tx(); @@ -666,7 +717,7 @@ mod tests { source_id: "test-source".into(), shard_id: 2, mrecord_batch: Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc-223\0\x01"), + mrecord_buffer: Bytes::from_static(b"\0\0test-doc-223\0\x02"), mrecord_lengths: vec![14, 2], }), from_position_exclusive: Some(22u64.into()), @@ -703,6 +754,30 @@ mod tests { assert_eq!(partition_deltas[1].0, 2u64.into()); assert_eq!(partition_deltas[1].1.from, Position::from(22u64)); assert_eq!(partition_deltas[1].1.to, Position::Eof); + + source + .emit_batches(&doc_processor_mailbox, &ctx) + .await + .unwrap(); + let shard = source.assigned_shards.get(&2).unwrap(); + assert_eq!(shard.status, IndexingStatus::Complete); + + fetch_response_tx + .send(Err(FetchStreamError { + index_uid: "test-index:0".into(), + source_id: "test-source".into(), + shard_id: 1, + ingest_error: IngestV2Error::Internal("test-error".to_string()), + })) + .await + .unwrap(); + + source + .emit_batches(&doc_processor_mailbox, &ctx) + .await + .unwrap(); + let shard = source.assigned_shards.get(&1).unwrap(); + assert_eq!(shard.status, IndexingStatus::Error); } #[tokio::test] @@ -808,6 +883,7 @@ mod tests { follower_id_opt: None, partition_id: 1u64.into(), current_position_inclusive: 11u64.into(), + status: IndexingStatus::Active, }, ); source.assigned_shards.insert( @@ -817,6 +893,7 @@ mod tests { follower_id_opt: Some("test-ingester-1".into()), partition_id: 2u64.into(), current_position_inclusive: 22u64.into(), + status: IndexingStatus::Active, }, ); source.assigned_shards.insert( @@ -826,6 +903,7 @@ mod tests { follower_id_opt: Some("test-ingester-0".into()), partition_id: 3u64.into(), current_position_inclusive: 33u64.into(), + status: IndexingStatus::Active, }, ); source.assigned_shards.insert( @@ -835,6 +913,7 @@ mod tests { follower_id_opt: Some("test-ingester-3".into()), partition_id: 4u64.into(), current_position_inclusive: 44u64.into(), + status: IndexingStatus::Active, }, ); source.assigned_shards.insert( @@ -844,6 +923,7 @@ mod tests { follower_id_opt: Some("test-ingester-3".into()), partition_id: 4u64.into(), current_position_inclusive: Position::Beginning, + status: IndexingStatus::Active, }, ); let checkpoint = SourceCheckpoint::from_iter(vec![ diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index ec8807efbff..f9dcbb39f41 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -36,6 +36,7 @@ quickwit-config = { workspace = true } quickwit-proto = { workspace = true } [dev-dependencies] +itertools = { workspace = true } mockall = { workspace = true } rand = { workspace = true } rand_distr = { workspace = true } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index 40d0d3ae37e..9ace7de58f1 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -214,6 +214,14 @@ impl FetchTask { } } +#[derive(Debug)] +pub struct FetchStreamError { + pub index_uid: IndexUid, + pub source_id: SourceId, + pub shard_id: ShardId, + pub ingest_error: IngestV2Error, +} + /// Combines multiple fetch streams originating from different ingesters into a single stream. It /// tolerates the failure of ingesters and automatically fails over to replica shards. pub struct MultiFetchStream { @@ -221,8 +229,8 @@ pub struct MultiFetchStream { client_id: ClientId, ingester_pool: IngesterPool, fetch_task_handles: HashMap>, - fetch_response_rx: mpsc::Receiver>, - fetch_response_tx: mpsc::Sender>, + fetch_response_rx: mpsc::Receiver>, + fetch_response_tx: mpsc::Sender>, } impl MultiFetchStream { @@ -239,7 +247,7 @@ impl MultiFetchStream { } #[cfg(any(test, feature = "testsuite"))] - pub fn fetch_response_tx(&self) -> mpsc::Sender> { + pub fn fetch_response_tx(&self) -> mpsc::Sender> { self.fetch_response_tx.clone() } @@ -307,7 +315,7 @@ impl MultiFetchStream { /// # Cancel safety /// /// This method is cancel safe. - pub async fn next(&mut self) -> IngestV2Result { + pub async fn next(&mut self) -> Result { // Because we always hold a sender and never call `close()` on the receiver, the channel is // always open. self.fetch_response_rx @@ -363,7 +371,7 @@ async fn fault_tolerant_fetch_task( to_position_inclusive: Position, ingester_ids: Vec, ingester_pool: IngesterPool, - fetch_response_tx: mpsc::Sender>, + fetch_response_tx: mpsc::Sender>, ) { // TODO: We can probably simplify this code by breaking it into smaller functions. 'outer: for (ingester_idx, ingester_id) in ingester_ids.iter().enumerate() { @@ -393,7 +401,14 @@ async fn fault_tolerant_fetch_task( }; // Attempt to send the error to the consumer in a best-effort manner before // returning. - let _ = fetch_response_tx.send(Err(ingest_error)).await; + let fetch_stream_error = FetchStreamError { + index_uid, + source_id, + shard_id, + ingest_error, + }; + let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + return; } continue; } @@ -427,7 +442,14 @@ async fn fault_tolerant_fetch_task( error=%ingest_error, "failed to open fetch stream from ingester `{ingester_id}`: closing fetch stream" ); - let _ = fetch_response_tx.send(Err(ingest_error)).await; + let fetch_stream_error = FetchStreamError { + index_uid, + source_id, + shard_id, + ingest_error, + }; + let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + return; } continue; } @@ -466,7 +488,14 @@ async fn fault_tolerant_fetch_task( error=%ingest_error, "failed to fetch records from ingester `{ingester_id}`: closing fetch stream" ); - let _ = fetch_response_tx.send(Err(ingest_error)).await; + let fetch_stream_error = FetchStreamError { + index_uid, + source_id, + shard_id, + ingest_error, + }; + let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + return; } continue 'outer; } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingest.md b/quickwit/quickwit-ingest/src/ingest_v2/ingest.md index 85bdf8d2d15..6e501a24ed4 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingest.md +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingest.md @@ -2,7 +2,7 @@ ### Settings -- ingest request timeout (30s), `Itimeout` +- ingest request timeout (35s), `Itimeout` - persist request timeout (6s), `Ptimeout` - replicate request timeout (3s), `Rtimeout` - number of persist attempts (5), `k` diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index d6b27aaf352..361b9fb7852 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -267,10 +267,11 @@ impl IngesterService for Ingester { if shard.is_closed() { let persist_failure = PersistFailure { + subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - failure_reason: PersistFailureReason::ShardClosed as i32, + reason: PersistFailureReason::ShardClosed as i32, }; persist_failures.push(persist_failure); continue; @@ -312,6 +313,7 @@ impl IngesterService for Ingester { if let Some(follower_id) = follower_id_opt { let replicate_subrequest = ReplicateSubrequest { + subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, @@ -325,6 +327,7 @@ impl IngesterService for Ingester { .push(replicate_subrequest); } else { let persist_success = PersistSuccess { + subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, @@ -377,6 +380,7 @@ impl IngesterService for Ingester { }; for replicate_success in replicate_response.successes { let persist_success = PersistSuccess { + subrequest_id: replicate_success.subrequest_id, index_uid: replicate_success.index_uid, source_id: replicate_success.source_id, shard_id: replicate_success.shard_id, @@ -693,6 +697,7 @@ mod tests { commit_type: CommitTypeV2::Force as i32, subrequests: vec![ PersistSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -700,6 +705,7 @@ mod tests { doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), }, PersistSubrequest { + subrequest_id: 1, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -822,6 +828,7 @@ mod tests { commit_type: CommitTypeV2::Force as i32, subrequests: vec![ PersistSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -829,6 +836,7 @@ mod tests { doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), }, PersistSubrequest { + subrequest_id: 1, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -972,6 +980,7 @@ mod tests { commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ PersistSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -979,6 +988,7 @@ mod tests { doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), }, PersistSubrequest { + subrequest_id: 1, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -1066,6 +1076,7 @@ mod tests { commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ PersistSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -1073,6 +1084,7 @@ mod tests { doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), }, PersistSubrequest { + subrequest_id: 1, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, @@ -1116,6 +1128,7 @@ mod tests { leader_id: self_node_id.to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![PersistSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index 53ab623fbfb..16e4317ab70 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -26,12 +26,13 @@ mod router; mod shard_table; #[cfg(test)] mod test_utils; +mod workbench; use quickwit_common::tower::Pool; use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::types::NodeId; -pub use self::fetch::MultiFetchStream; +pub use self::fetch::{FetchStreamError, MultiFetchStream}; pub use self::ingester::Ingester; pub use self::mrecord::{decoded_mrecords, MRecord}; pub use self::router::IngestRouter; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index ede0979f181..e6fe41bbe68 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -394,6 +394,7 @@ impl ReplicationTask { replica_shard.set_replication_position_inclusive(current_position_inclusive.clone()); let replicate_success = ReplicateSuccess { + subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, @@ -490,6 +491,7 @@ mod tests { .subrequests .iter() .map(|subrequest| ReplicateSuccess { + subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid.clone(), source_id: subrequest.source_id.clone(), shard_id: subrequest.shard_id, @@ -519,6 +521,7 @@ mod tests { commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ ReplicateSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -527,6 +530,7 @@ mod tests { to_position_inclusive: Some(Position::from(0u64)), }, ReplicateSubrequest { + subrequest_id: 1, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, @@ -535,6 +539,7 @@ mod tests { to_position_inclusive: Some(Position::from(1u64)), }, ReplicateSubrequest { + subrequest_id: 2, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -637,6 +642,7 @@ mod tests { commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ ReplicateSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -645,6 +651,7 @@ mod tests { to_position_inclusive: Some(Position::from(0u64)), }, ReplicateSubrequest { + subrequest_id: 1, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, @@ -653,6 +660,7 @@ mod tests { to_position_inclusive: Some(Position::from(1u64)), }, ReplicateSubrequest { + subrequest_id: 2, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -724,6 +732,7 @@ mod tests { follower_id: "test-follower".to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ReplicateSubrequest { + subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index f05da8feb9d..0cd74837ecb 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -35,12 +35,14 @@ use quickwit_proto::ingest::ingester::{ use quickwit_proto::ingest::router::{ IngestRequestV2, IngestResponseV2, IngestRouterService, IngestSubrequest, }; -use quickwit_proto::ingest::{IngestV2Error, IngestV2Result}; +use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result}; use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceId}; use tokio::sync::RwLock; +use tracing::warn; use super::ingester::PERSIST_REQUEST_TIMEOUT; use super::shard_table::ShardTable; +use super::workbench::IngestWorkbench; use super::IngesterPool; /// Duration after which ingest requests time out with [`IngestV2Error::Timeout`]. @@ -50,6 +52,8 @@ pub(super) const INGEST_REQUEST_TIMEOUT: Duration = if cfg!(any(test, feature = Duration::from_secs(35) }; +const MAX_PERSIST_ATTEMPTS: usize = 5; + type LeaderId = String; #[derive(Clone)] @@ -100,11 +104,12 @@ impl IngestRouter { /// [`GetOrCreateOpenShardsRequest`] request if open shards do not exist for all the them. async fn make_get_or_create_open_shard_request( &self, - subrequests: &[IngestSubrequest], + subrequests: impl Iterator, ingester_pool: &IngesterPool, ) -> GetOrCreateOpenShardsRequest { let state_guard = self.state.read().await; let mut get_open_shards_subrequests = Vec::new(); + // `closed_shards` and `unavailable_leaders` are populated by calls to `has_open_shards` // as we're looking for open shards to route the subrequests to. let mut closed_shards: Vec = Vec::new(); @@ -125,7 +130,6 @@ impl IngestRouter { get_open_shards_subrequests.push(subrequest); } } - GetOrCreateOpenShardsRequest { subrequests: get_open_shards_subrequests, closed_shards, @@ -161,6 +165,7 @@ impl IngestRouter { async fn process_persist_results( &mut self, + workbench: &mut IngestWorkbench, mut persist_futures: FuturesUnordered< impl Future>, >, @@ -170,15 +175,19 @@ impl IngestRouter { while let Some(persist_result) = persist_futures.next().await { match persist_result { Ok(persist_response) => { + for persist_success in persist_response.successes { + workbench.record_success(persist_success); + } for persist_failure in persist_response.failures { - if persist_failure.failure_reason() == PersistFailureReason::ShardClosed { - let index_uid: IndexUid = persist_failure.index_uid.into(); - let source_id: SourceId = persist_failure.source_id; + if persist_failure.reason() == PersistFailureReason::ShardClosed { + let index_uid: IndexUid = persist_failure.index_uid.clone().into(); + let source_id: SourceId = persist_failure.source_id.clone(); closed_shards .entry((index_uid, source_id)) .or_default() .push(persist_failure.shard_id); } + workbench.record_failure(persist_failure); } } Err(_persist_error) => { @@ -197,15 +206,25 @@ impl IngestRouter { } } - async fn ingest_attempt_one( - &mut self, - ingest_request: IngestRequestV2, - ) -> IngestV2Result { + async fn batch_persist(&mut self, workbench: &mut IngestWorkbench, commit_type: CommitTypeV2) { let get_or_create_open_shards_request = self - .make_get_or_create_open_shard_request(&ingest_request.subrequests, &self.ingester_pool) + .make_get_or_create_open_shard_request( + workbench.pending_subrequests(), + &self.ingester_pool, + ) .await; - self.populate_shard_table(get_or_create_open_shards_request) - .await?; + + if let Err(error) = self + .populate_shard_table(get_or_create_open_shards_request) + .await + { + warn!( + "failed to obtain open shards from control plane: `{}`", + error + ); + } + // List of subrequest IDs for which no shards were available to route the subrequests to. + let mut unavailable_subrequest_ids = Vec::new(); let mut per_leader_persist_subrequests: HashMap<&LeaderId, Vec> = HashMap::new(); @@ -215,20 +234,23 @@ impl IngestRouter { // TODO: Here would be the most optimal place to split the body of the HTTP request into // lines, validate, transform and then pack the docs into compressed batches routed // to the right shards. - for ingest_subrequest in ingest_request.subrequests { - let shard = state_guard - .shard_table - .find_entry(&ingest_subrequest.index_id, &ingest_subrequest.source_id) - .expect("TODO") - .next_open_shard_round_robin(&self.ingester_pool) - .expect("TODO"); + for subrequest in workbench.pending_subrequests() { + let Some(shard) = state_guard + .shard_table + .find_entry(&subrequest.index_id, &subrequest.source_id) + .and_then(|entry| entry.next_open_shard_round_robin(&self.ingester_pool)) + else { + unavailable_subrequest_ids.push(subrequest.subrequest_id); + continue; + }; let persist_subrequest = PersistSubrequest { + subrequest_id: subrequest.subrequest_id, index_uid: shard.index_uid.clone(), - source_id: ingest_subrequest.source_id, + source_id: subrequest.source_id.clone(), shard_id: shard.shard_id, follower_id: shard.follower_id.clone(), - doc_batch: ingest_subrequest.doc_batch, + doc_batch: subrequest.doc_batch.clone(), }; per_leader_persist_subrequests .entry(&shard.leader_id) @@ -239,12 +261,17 @@ impl IngestRouter { for (leader_id, subrequests) in per_leader_persist_subrequests { let leader_id: NodeId = leader_id.clone().into(); - let mut ingester = self.ingester_pool.get(&leader_id).expect("TODO"); - + let Some(mut ingester) = self.ingester_pool.get(&leader_id) else { + let subrequest_ids = subrequests + .iter() + .map(|subrequest| subrequest.subrequest_id); + unavailable_subrequest_ids.extend(subrequest_ids); + continue; + }; let persist_request = PersistRequest { leader_id: leader_id.into(), subrequests, - commit_type: ingest_request.commit_type, + commit_type: commit_type as i32, }; let persist_future = async move { tokio::time::timeout(PERSIST_REQUEST_TIMEOUT, ingester.persist(persist_request)) @@ -255,20 +282,28 @@ impl IngestRouter { } drop(state_guard); - self.process_persist_results(persist_futures).await; - - Ok(IngestResponseV2 { - subresponses: Vec::new(), - }) + for subrequest_id in unavailable_subrequest_ids { + workbench.record_no_shards_available(subrequest_id); + } + self.process_persist_results(workbench, persist_futures) + .await; } - async fn ingest_attempt_many( + async fn retry_batch_persist( &mut self, ingest_request: IngestRequestV2, + max_num_attempts: usize, ) -> IngestV2Result { - // TODO: Introduce IngestRequestWorbench and implement retry logic and perform multiple - // attempts. - self.ingest_attempt_one(ingest_request).await + let mut num_attempts = 0; + + let commit_type = ingest_request.commit_type(); + let mut workbench = IngestWorkbench::new(ingest_request.subrequests); + + while !workbench.is_complete() && num_attempts < max_num_attempts { + self.batch_persist(&mut workbench, commit_type).await; + num_attempts += 1; + } + workbench.into_ingest_response() } async fn ingest_timeout( @@ -276,9 +311,12 @@ impl IngestRouter { ingest_request: IngestRequestV2, timeout_duration: Duration, ) -> IngestV2Result { - tokio::time::timeout(timeout_duration, self.ingest_attempt_many(ingest_request)) - .await - .map_err(|_| IngestV2Error::Timeout)? + tokio::time::timeout( + timeout_duration, + self.retry_batch_persist(ingest_request, MAX_PERSIST_ATTEMPTS), + ) + .await + .map_err(|_| IngestV2Error::Timeout)? } } @@ -295,14 +333,16 @@ impl IngestRouterService for IngestRouter { #[cfg(test)] mod tests { + use std::iter; use std::sync::atomic::AtomicUsize; use quickwit_proto::control_plane::{GetOpenShardsSubresponse, GetOrCreateOpenShardsResponse}; use quickwit_proto::ingest::ingester::{ - IngesterServiceClient, PersistFailure, PersistResponse, + IngesterServiceClient, PersistFailure, PersistResponse, PersistSuccess, }; use quickwit_proto::ingest::router::IngestSubrequest; use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard, ShardState}; + use quickwit_proto::types::Position; use super::*; use crate::ingest_v2::shard_table::ShardTableEntry; @@ -320,7 +360,7 @@ mod tests { replication_factor, ); let get_or_create_open_shard_request = router - .make_get_or_create_open_shard_request(&[], &ingester_pool) + .make_get_or_create_open_shard_request(iter::empty(), &ingester_pool) .await; assert!(get_or_create_open_shard_request.subrequests.is_empty()); @@ -358,18 +398,20 @@ mod tests { let ingest_subrequests = [ IngestSubrequest { + subrequest_id: 0, index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), ..Default::default() }, IngestSubrequest { + subrequest_id: 1, index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), ..Default::default() }, ]; let get_or_create_open_shard_request = router - .make_get_or_create_open_shard_request(&ingest_subrequests, &ingester_pool) + .make_get_or_create_open_shard_request(ingest_subrequests.iter(), &ingester_pool) .await; assert_eq!(get_or_create_open_shard_request.subrequests.len(), 2); @@ -406,7 +448,7 @@ mod tests { ); let get_or_create_open_shard_request = router - .make_get_or_create_open_shard_request(&ingest_subrequests, &ingester_pool) + .make_get_or_create_open_shard_request(ingest_subrequests.iter(), &ingester_pool) .await; let subrequest = &get_or_create_open_shard_request.subrequests[0]; @@ -450,6 +492,7 @@ mod tests { source_id: "test-source".to_string(), open_shards: vec![Shard { shard_id: 1, + shard_state: ShardState::Open as i32, ..Default::default() }], }, @@ -459,10 +502,12 @@ mod tests { open_shards: vec![ Shard { shard_id: 1, + shard_state: ShardState::Open as i32, ..Default::default() }, Shard { shard_id: 2, + shard_state: ShardState::Open as i32, ..Default::default() }, ], @@ -551,12 +596,14 @@ mod tests { vec![Shard { index_uid: "test-index-0:0".to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() }], ); drop(state_guard); + let mut workbench = IngestWorkbench::new(Vec::new()); let persist_futures = FuturesUnordered::new(); persist_futures.push(async { @@ -564,14 +611,17 @@ mod tests { leader_id: "test-ingester-0".to_string(), successes: Vec::new(), failures: vec![PersistFailure { + subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - failure_reason: PersistFailureReason::ShardClosed as i32, + reason: PersistFailureReason::ShardClosed as i32, }], }) }); - router.process_persist_results(persist_futures).await; + router + .process_persist_results(&mut workbench, persist_futures) + .await; let state_guard = router.state.read().await; let shard_table_entry = state_guard @@ -600,29 +650,32 @@ mod tests { ); let mut state_guard = router.state.write().await; state_guard.shard_table.insert_shards( - "test-index-0", + "test-index-0:0", "test-source", vec![Shard { index_uid: "test-index-0:0".to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() }], ); state_guard.shard_table.insert_shards( - "test-index-1", + "test-index-1:0", "test-source", vec![ Shard { - index_uid: "test-index-1:1".to_string(), + index_uid: "test-index-1:0".to_string(), shard_id: 1, + shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), follower_id: Some("test-ingester-1".to_string()), ..Default::default() }, Shard { - index_uid: "test-index-1:1".to_string(), + index_uid: "test-index-1:0".to_string(), shard_id: 2, + shard_state: ShardState::Open as i32, leader_id: "test-ingester-1".to_string(), follower_id: Some("test-ingester-2".to_string()), ..Default::default() @@ -641,6 +694,7 @@ mod tests { assert_eq!(request.commit_type(), CommitTypeV2::Auto); let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); assert_eq!(subrequest.shard_id, 1); @@ -651,7 +705,8 @@ mod tests { ); let subrequest = &request.subrequests[1]; - assert_eq!(subrequest.index_uid, "test-index-1:1"); + assert_eq!(subrequest.subrequest_id, 1); + assert_eq!(subrequest.index_uid, "test-index-1:0"); assert_eq!(subrequest.source_id, "test-source"); assert_eq!(subrequest.shard_id, 1); assert_eq!(subrequest.follower_id(), "test-ingester-1"); @@ -662,7 +717,22 @@ mod tests { let response = PersistResponse { leader_id: request.leader_id, - successes: Vec::new(), + successes: vec![ + PersistSuccess { + subrequest_id: 0, + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + replication_position_inclusive: Some(Position::from(1u64)), + }, + PersistSuccess { + subrequest_id: 1, + index_uid: "test-index-1:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + replication_position_inclusive: Some(Position::from(0u64)), + }, + ], failures: Vec::new(), }; Ok(response) @@ -676,6 +746,7 @@ mod tests { assert_eq!(request.commit_type(), CommitTypeV2::Auto); let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); assert_eq!(subrequest.shard_id, 1); @@ -687,7 +758,13 @@ mod tests { let response = PersistResponse { leader_id: request.leader_id, - successes: Vec::new(), + successes: vec![PersistSuccess { + subrequest_id: 0, + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + replication_position_inclusive: Some(Position::from(3u64)), + }], failures: Vec::new(), }; Ok(response) @@ -705,7 +782,8 @@ mod tests { assert_eq!(request.commit_type(), CommitTypeV2::Auto); let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.index_uid, "test-index-1:1"); + assert_eq!(subrequest.subrequest_id, 1); + assert_eq!(subrequest.index_uid, "test-index-1:0"); assert_eq!(subrequest.source_id, "test-source"); assert_eq!(subrequest.shard_id, 2); assert_eq!(subrequest.follower_id(), "test-ingester-2"); @@ -716,7 +794,13 @@ mod tests { let response = PersistResponse { leader_id: request.leader_id, - successes: Vec::new(), + successes: vec![PersistSuccess { + subrequest_id: 1, + index_uid: "test-index-1:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 2, + replication_position_inclusive: Some(Position::from(0u64)), + }], failures: Vec::new(), }; Ok(response) @@ -727,11 +811,13 @@ mod tests { let ingest_request = IngestRequestV2 { subrequests: vec![ IngestSubrequest { + subrequest_id: 0, index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["test-doc-foo", "test-doc-bar"])), }, IngestSubrequest { + subrequest_id: 1, index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["test-doc-qux"])), @@ -744,11 +830,13 @@ mod tests { let ingest_request = IngestRequestV2 { subrequests: vec![ IngestSubrequest { + subrequest_id: 0, index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["test-doc-moo", "test-doc-baz"])), }, IngestSubrequest { + subrequest_id: 1, index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["test-doc-tux"])), @@ -758,4 +846,110 @@ mod tests { }; router.ingest(ingest_request).await.unwrap(); } + + #[tokio::test] + async fn test_router_ingest_retry() { + let self_node_id = "test-router".into(); + let control_plane = ControlPlaneServiceClient::mock().into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + let mut router = IngestRouter::new( + self_node_id, + control_plane, + ingester_pool.clone(), + replication_factor, + ); + let mut state_guard = router.state.write().await; + state_guard.shard_table.insert_shards( + "test-index-0:0", + "test-source", + vec![Shard { + index_uid: "test-index-0:0".to_string(), + shard_id: 1, + shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), + ..Default::default() + }], + ); + drop(state_guard); + + let mut ingester_mock_0 = IngesterServiceClient::mock(); + ingester_mock_0 + .expect_persist() + .once() + .returning(|request| { + assert_eq!(request.leader_id, "test-ingester-0"); + assert_eq!(request.subrequests.len(), 1); + assert_eq!(request.commit_type(), CommitTypeV2::Auto); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); + assert_eq!(subrequest.index_uid, "test-index-0:0"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, 1); + assert!(subrequest.follower_id.is_none()); + assert_eq!( + subrequest.doc_batch, + Some(DocBatchV2::for_test(["test-doc-foo"])) + ); + + let response = PersistResponse { + leader_id: request.leader_id, + successes: Vec::new(), + failures: vec![PersistFailure { + subrequest_id: 0, + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + reason: PersistFailureReason::RateLimited as i32, + }], + }; + Ok(response) + }); + ingester_mock_0 + .expect_persist() + .once() + .returning(|request| { + assert_eq!(request.leader_id, "test-ingester-0"); + assert_eq!(request.subrequests.len(), 1); + assert_eq!(request.commit_type(), CommitTypeV2::Auto); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); + assert_eq!(subrequest.index_uid, "test-index-0:0"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, 1); + assert!(subrequest.follower_id.is_none()); + assert_eq!( + subrequest.doc_batch, + Some(DocBatchV2::for_test(["test-doc-foo"])) + ); + + let response = PersistResponse { + leader_id: request.leader_id, + successes: vec![PersistSuccess { + subrequest_id: 0, + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + replication_position_inclusive: Some(Position::from(0u64)), + }], + failures: Vec::new(), + }; + Ok(response) + }); + let ingester_0: IngesterServiceClient = ingester_mock_0.into(); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + + let ingest_request = IngestRequestV2 { + subrequests: vec![IngestSubrequest { + subrequest_id: 0, + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), + }], + commit_type: CommitTypeV2::Auto as i32, + }; + router.ingest(ingest_request).await.unwrap(); + } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs new file mode 100644 index 00000000000..018cc246f56 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -0,0 +1,279 @@ +// 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::collections::HashMap; + +use quickwit_proto::ingest::ingester::{PersistFailure, PersistSuccess}; +use quickwit_proto::ingest::router::{ + IngestFailure, IngestFailureReason, IngestResponseV2, IngestSubrequest, IngestSuccess, +}; +use quickwit_proto::ingest::IngestV2Result; +use quickwit_proto::types::SubrequestId; +use tracing::warn; + +/// A helper struct for managing the state of the subrequests of an ingest request during multiple +/// persist attempts. +pub(super) struct IngestWorkbench { + subworkbenches: HashMap, + num_successes: usize, +} + +impl IngestWorkbench { + pub fn new(subrequests: Vec) -> Self { + let subworkbenches = subrequests + .into_iter() + .map(|subrequest| { + ( + subrequest.subrequest_id, + IngestSubworkbench::new(subrequest), + ) + }) + .collect(); + + Self { + subworkbenches, + num_successes: 0, + } + } + + #[cfg(not(test))] + pub fn pending_subrequests(&self) -> impl Iterator { + self.subworkbenches.values().filter_map(|subworbench| { + if !subworbench.is_success() { + Some(&subworbench.subrequest) + } else { + None + } + }) + } + + pub fn record_success(&mut self, persist_success: PersistSuccess) { + let Some(subworkbench) = self.subworkbenches.get_mut(&persist_success.subrequest_id) else { + warn!( + "could not find subrequest `{}` in workbench", + persist_success.subrequest_id + ); + return; + }; + self.num_successes += 1; + subworkbench.num_attempts += 1; + subworkbench.persist_success_opt = Some(persist_success); + } + + pub fn record_failure(&mut self, persist_failure: PersistFailure) { + let Some(subworkbench) = self.subworkbenches.get_mut(&persist_failure.subrequest_id) else { + warn!( + "could not find subrequest `{}` in workbench", + persist_failure.subrequest_id + ); + return; + }; + subworkbench.num_attempts += 1; + subworkbench.last_attempt_failure_opt = Some(SubworkbenchFailure::Persist(persist_failure)); + } + + pub fn record_no_shards_available(&mut self, subrequest_id: SubrequestId) { + let Some(subworkbench) = self.subworkbenches.get_mut(&subrequest_id) else { + warn!("could not find subrequest `{}` in workbench", subrequest_id); + return; + }; + subworkbench.num_attempts += 1; + subworkbench.last_attempt_failure_opt = Some(SubworkbenchFailure::NoShardsAvailable); + } + + pub fn is_complete(&self) -> bool { + self.num_successes == self.subworkbenches.len() + } + + pub fn into_ingest_response(self) -> IngestV2Result { + let num_subworkbenches = self.subworkbenches.len(); + let mut successes = Vec::with_capacity(self.num_successes); + let mut failures = Vec::with_capacity(num_subworkbenches - self.num_successes); + + for subworkbench in self.subworkbenches.into_values() { + if let Some(persist_success) = subworkbench.persist_success_opt { + let success = IngestSuccess { + subrequest_id: persist_success.subrequest_id, + index_uid: persist_success.index_uid, + source_id: persist_success.source_id, + shard_id: persist_success.shard_id, + replication_position_inclusive: persist_success.replication_position_inclusive, + }; + successes.push(success); + } else if let Some(failure) = subworkbench.last_attempt_failure_opt { + let failure = IngestFailure { + subrequest_id: subworkbench.subrequest.subrequest_id, + index_id: subworkbench.subrequest.index_id, + source_id: subworkbench.subrequest.source_id, + reason: failure.reason() as i32, + }; + failures.push(failure); + } + } + assert!(successes.len() + failures.len() == num_subworkbenches); + + Ok(IngestResponseV2 { + successes, + failures, + }) + } + + #[cfg(test)] + pub fn pending_subrequests(&self) -> impl Iterator { + use itertools::Itertools; + + self.subworkbenches + .values() + .filter_map(|subworbench| { + if !subworbench.is_success() { + Some(&subworbench.subrequest) + } else { + None + } + }) + .sorted_by_key(|subrequest| subrequest.subrequest_id) + } +} + +#[derive(Debug)] +enum SubworkbenchFailure { + NoShardsAvailable, + Persist(PersistFailure), +} + +impl SubworkbenchFailure { + fn reason(&self) -> IngestFailureReason { + // TODO: Return a better failure reason for `Self::Persist`. + match self { + Self::NoShardsAvailable => IngestFailureReason::NoShardsAvailable, + Self::Persist(_) => IngestFailureReason::Unspecified, + } + } +} + +pub(super) struct IngestSubworkbench { + subrequest: IngestSubrequest, + persist_success_opt: Option, + last_attempt_failure_opt: Option, + num_attempts: usize, +} + +impl IngestSubworkbench { + pub fn new(subrequest: IngestSubrequest) -> Self { + Self { + subrequest, + persist_success_opt: None, + last_attempt_failure_opt: None, + num_attempts: 0, + } + } + + pub fn is_success(&self) -> bool { + self.persist_success_opt.is_some() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_ingest_subworkbench() { + let subrequest = IngestSubrequest { + ..Default::default() + }; + let mut subworkbench = IngestSubworkbench::new(subrequest); + assert!(!subworkbench.is_success()); + + let persist_success = PersistSuccess { + ..Default::default() + }; + subworkbench.persist_success_opt = Some(persist_success); + assert!(subworkbench.is_success()); + } + + #[test] + fn test_ingest_workbench() { + let subrequests = vec![ + IngestSubrequest { + subrequest_id: 0, + ..Default::default() + }, + IngestSubrequest { + subrequest_id: 1, + ..Default::default() + }, + ]; + let mut workbench = IngestWorkbench::new(subrequests); + assert_eq!(workbench.pending_subrequests().count(), 2); + assert!(!workbench.is_complete()); + + let persist_success = PersistSuccess { + subrequest_id: 0, + ..Default::default() + }; + workbench.record_success(persist_success); + + assert_eq!(workbench.num_successes, 1); + assert_eq!(workbench.pending_subrequests().count(), 1); + assert_eq!( + workbench + .pending_subrequests() + .next() + .unwrap() + .subrequest_id, + 1 + ); + + let subworkbench = workbench.subworkbenches.get(&0).unwrap(); + assert_eq!(subworkbench.num_attempts, 1); + assert!(subworkbench.is_success()); + + let persist_failure = PersistFailure { + subrequest_id: 1, + ..Default::default() + }; + workbench.record_failure(persist_failure); + + assert_eq!(workbench.num_successes, 1); + assert_eq!(workbench.pending_subrequests().count(), 1); + assert_eq!( + workbench + .pending_subrequests() + .next() + .unwrap() + .subrequest_id, + 1 + ); + + let subworkbench = workbench.subworkbenches.get(&1).unwrap(); + assert_eq!(subworkbench.num_attempts, 1); + assert!(subworkbench.last_attempt_failure_opt.is_some()); + + let persist_success = PersistSuccess { + subrequest_id: 1, + ..Default::default() + }; + workbench.record_success(persist_success); + + assert!(workbench.is_complete()); + assert_eq!(workbench.num_successes, 2); + assert_eq!(workbench.pending_subrequests().count(), 0); + } +} diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs index 821164dd0f9..79c9949814e 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs @@ -110,14 +110,19 @@ impl Shards { ) -> MetastoreResult> { let mut mutation_occurred = false; - if subrequest.next_shard_id + 1 < self.next_shard_id - || subrequest.next_shard_id > self.next_shard_id - { + // When a response is lost, the control plane can "lag by one shard ID". + // The inconsistency should be resolved on the next attempt. + // + // `self.next_shard_id - 1` does not underflow because `self.next_shard_id` is always > 0. + let ok_next_shard_ids = self.next_shard_id - 1..=self.next_shard_id; + + if !ok_next_shard_ids.contains(&subrequest.next_shard_id) { warn!( - "control plane and metastore next shard IDs do not match, expected `{}`, got `{}`", + "control plane state is inconsistent with that of the metastore, expected next \ + shard ID `{}`, got `{}`", self.next_shard_id, subrequest.next_shard_id - ) - // TODO: Return an error and crash the control plane. + ); + return Err(MetastoreError::InconsistentControlPlaneState); } let entry = self.shards.entry(subrequest.next_shard_id); @@ -128,13 +133,15 @@ impl Shards { index_uid: self.index_uid.clone().into(), source_id: self.source_id.clone(), shard_id: self.next_shard_id, + shard_state: ShardState::Open as i32, leader_id: subrequest.leader_id.clone(), follower_id: subrequest.follower_id.clone(), - ..Default::default() + publish_position_inclusive: Some(Position::Beginning), + publish_token: None, }; mutation_occurred = true; entry.insert(shard.clone()); - self.next_shard_id += 1; + self.next_shard_id = subrequest.next_shard_id + 1; info!( index_id=%self.index_uid.index_id(), @@ -147,13 +154,13 @@ impl Shards { shard } }; - let open_shards = vec![shard]; + let opened_shards = vec![shard]; let next_shard_id = self.next_shard_id; let response = OpenShardsSubresponse { index_uid: subrequest.index_uid, source_id: subrequest.source_id, - open_shards, + opened_shards, next_shard_id, }; if mutation_occurred { @@ -340,26 +347,28 @@ mod tests { }; assert_eq!(subresponse.index_uid, index_uid.as_str()); assert_eq!(subresponse.source_id, source_id); - assert_eq!(subresponse.open_shards.len(), 1); + assert_eq!(subresponse.opened_shards.len(), 1); - let shard = &subresponse.open_shards[0]; + let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, index_uid.as_str()); assert_eq!(shard.source_id, source_id); assert_eq!(shard.shard_id, 1); - assert_eq!(shard.shard_state, 0); + assert_eq!(shard.shard_state(), ShardState::Open); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id, None); assert_eq!(shard.publish_position_inclusive(), Position::Beginning); assert_eq!(shards.shards.get(&1).unwrap(), shard); + assert_eq!(shards.next_shard_id, 2); let MutationOccurred::No(subresponse) = shards.open_shards(subrequest).unwrap() else { panic!("Expected `MutationOccured::No`"); }; - assert_eq!(subresponse.open_shards.len(), 1); + assert_eq!(subresponse.opened_shards.len(), 1); - let shard = &subresponse.open_shards[0]; + let shard = &subresponse.opened_shards[0]; assert_eq!(shards.shards.get(&1).unwrap(), shard); + assert_eq!(shards.next_shard_id, 2); let subrequest = OpenShardsSubrequest { index_uid: index_uid.clone().into(), @@ -373,18 +382,29 @@ mod tests { }; assert_eq!(subresponse.index_uid, index_uid.as_str()); assert_eq!(subresponse.source_id, source_id); - assert_eq!(subresponse.open_shards.len(), 1); + assert_eq!(subresponse.opened_shards.len(), 1); - let shard = &subresponse.open_shards[0]; + let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, index_uid.as_str()); assert_eq!(shard.source_id, source_id); assert_eq!(shard.shard_id, 2); - assert_eq!(shard.shard_state, 0); + assert_eq!(shard.shard_state(), ShardState::Open); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id.as_ref().unwrap(), "follower_id"); assert_eq!(shard.publish_position_inclusive(), Position::Beginning); assert_eq!(shards.shards.get(&2).unwrap(), shard); + assert_eq!(shards.next_shard_id, 3); + + let subrequest = OpenShardsSubrequest { + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + leader_id: "leader_id".to_string(), + follower_id: Some("follower_id".to_string()), + next_shard_id: 1, + }; + let error = shards.open_shards(subrequest).unwrap_err(); + assert_eq!(error, MetastoreError::InconsistentControlPlaneState); } #[test] 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 3f6d780c3e2..78d9b7b9cb5 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -83,6 +83,7 @@ pub(crate) enum IndexState { Deleting, } +#[derive(Debug)] pub(crate) enum MutationOccurred { Yes(T), No(T), diff --git a/quickwit/quickwit-metastore/src/tests/shard.rs b/quickwit/quickwit-metastore/src/tests/shard.rs index c1cb710dcc3..87555664dbb 100644 --- a/quickwit/quickwit-metastore/src/tests/shard.rs +++ b/quickwit/quickwit-metastore/src/tests/shard.rs @@ -117,9 +117,9 @@ pub async fn test_metastore_open_shards< assert_eq!(subresponse.index_uid, test_index.index_uid.as_str()); assert_eq!(subresponse.source_id, test_index.source_config.source_id); assert_eq!(subresponse.next_shard_id, 2); - assert_eq!(subresponse.open_shards.len(), 1); + assert_eq!(subresponse.opened_shards.len(), 1); - let shard = &subresponse.open_shards[0]; + let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, test_index.index_uid.as_str()); assert_eq!(shard.source_id, test_index.source_config.source_id); assert_eq!(shard.shard_id, 1); @@ -138,9 +138,9 @@ pub async fn test_metastore_open_shards< assert_eq!(open_shards_response.subresponses.len(), 1); let subresponse = &open_shards_response.subresponses[0]; - assert_eq!(subresponse.open_shards.len(), 1); + assert_eq!(subresponse.opened_shards.len(), 1); - let shard = &subresponse.open_shards[0]; + let shard = &subresponse.opened_shards[0]; assert_eq!(shard.shard_id, 1); assert_eq!(shard.leader_id, "test-ingester-foo"); diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto index 9208f93a7ec..deea81a36e2 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingest.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -30,9 +30,10 @@ message Position { } enum CommitTypeV2 { - AUTO = 0; - WAIT = 1; - FORCE = 2; + COMMIT_TYPE_V2_UNSPECIFIED = 0; + COMMIT_TYPE_V2_AUTO = 1; + COMMIT_TYPE_V2_WAIT = 2; + COMMIT_TYPE_V2_FORCE = 3; } message DocBatchV2 { @@ -48,13 +49,14 @@ message MRecordBatch { } enum ShardState { + SHARD_STATE_UNSPECIFIED = 0; // The shard is open and accepts write requests. - OPEN = 0; + SHARD_STATE_OPEN = 1; // The ingester hosting the shard is unavailable. - UNAVAILABLE = 1; + SHARD_STATE_UNAVAILABLE = 2; // The shard is closed and cannot be written to. // It can be safely deleted if the publish position is superior or equal to the replication position. - CLOSED = 2; + SHARD_STATE_CLOSED = 3; } message Shard { diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 8d6e4ac229c..aca868f29b7 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -49,11 +49,12 @@ message PersistRequest { } message PersistSubrequest { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - optional string follower_id = 4; - quickwit.ingest.DocBatchV2 doc_batch = 5; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + optional string follower_id = 5; + quickwit.ingest.DocBatchV2 doc_batch = 6; } message PersistResponse { @@ -63,21 +64,26 @@ message PersistResponse { } message PersistSuccess { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - quickwit.ingest.Position replication_position_inclusive = 4; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + quickwit.ingest.Position replication_position_inclusive = 5; } enum PersistFailureReason { - SHARD_CLOSED = 0; + PERSIST_FAILURE_REASON_UNSPECIFIED = 0; + PERSIST_FAILURE_REASON_SHARD_CLOSED = 1; + PERSIST_FAILURE_REASON_RATE_LIMITED = 2; + PERSIST_FAILURE_REASON_RESOURCE_EXHAUSTED = 3; } message PersistFailure { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - PersistFailureReason failure_reason = 4; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + PersistFailureReason reason = 5; } message SynReplicationMessage { @@ -112,12 +118,13 @@ message ReplicateRequest { } message ReplicateSubrequest { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - quickwit.ingest.Position from_position_exclusive = 4; - quickwit.ingest.Position to_position_inclusive = 5; - ingest.DocBatchV2 doc_batch = 6; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + quickwit.ingest.Position from_position_exclusive = 5; + quickwit.ingest.Position to_position_inclusive = 6; + ingest.DocBatchV2 doc_batch = 7; } message ReplicateResponse { @@ -129,16 +136,18 @@ message ReplicateResponse { } message ReplicateSuccess { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - quickwit.ingest.Position replication_position_inclusive = 4; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + quickwit.ingest.Position replication_position_inclusive = 5; } message ReplicateFailure { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; // ingest.DocBatchV2 doc_batch = 4; // ingest.IngestError error = 5; } diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 0937e38861b..386ce457d3f 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -24,17 +24,18 @@ package quickwit.metastore; import "quickwit/ingest.proto"; enum SourceType { - CLI = 0; - FILE = 1; - GCP_PUBSUB = 2; - INGEST_V1 = 3; - INGEST_V2 = 4; - KAFKA = 5; - KINESIS = 6; - NATS = 7; - PULSAR = 8; - VEC = 9; - VOID = 10; + SOURCE_TYPE_UNSPECIFIED = 0; + SOURCE_TYPE_CLI = 1; + SOURCE_TYPE_FILE = 2; + SOURCE_TYPE_GCP_PUBSUB = 3; + SOURCE_TYPE_INGEST_V1 = 4; + SOURCE_TYPE_INGEST_V2 = 5; + SOURCE_TYPE_KAFKA = 6; + SOURCE_TYPE_KINESIS = 7; + SOURCE_TYPE_NATS = 8; + SOURCE_TYPE_PULSAR = 9; + SOURCE_TYPE_VEC = 10; + SOURCE_TYPE_VOID = 11; } service MetastoreService { @@ -275,7 +276,7 @@ message OpenShardsResponse { message OpenShardsSubresponse { string index_uid = 1; string source_id = 2; - repeated quickwit.ingest.Shard open_shards = 3; + repeated quickwit.ingest.Shard opened_shards = 3; uint64 next_shard_id = 4; } diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto index be25f6003fb..68bae11746e 100644 --- a/quickwit/quickwit-proto/protos/quickwit/router.proto +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -35,31 +35,37 @@ message IngestRequestV2 { } message IngestSubrequest { - string index_id = 1; - string source_id = 2; - quickwit.ingest.DocBatchV2 doc_batch = 3; + // The subrequest ID is used to identify the various subrequests and responses + // (ingest, persist, replicate) at play during the ingest and replication + // process. + uint32 subrequest_id = 1; + string index_id = 2; + string source_id = 3; + quickwit.ingest.DocBatchV2 doc_batch = 4; } message IngestResponseV2 { - repeated IngestSubresponse subresponses = 1; -} - -message IngestSubresponse { - string index_id = 1; - string source_id = 2; - repeated IngestSuccess successes = 3; - repeated IngestFailure failures = 4; + repeated IngestSuccess successes = 1; + repeated IngestFailure failures = 2; } message IngestSuccess { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; + uint32 subrequest_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; // Replication position inclusive. - quickwit.ingest.Position replication_position_inclusive = 4; + quickwit.ingest.Position replication_position_inclusive = 5; +} + +enum IngestFailureReason { + INGEST_FAILURE_REASON_UNSPECIFIED = 0; + INGEST_FAILURE_REASON_NO_SHARDS_AVAILABLE = 1; } message IngestFailure { - uint64 shard_id = 1; - // IngestFailureReason reason = 2; + uint32 subrequest_id = 1; + string index_id = 2; + string source_id = 3; + IngestFailureReason reason = 5; } 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 b060d386c82..72cc0d36a57 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -13,15 +13,17 @@ pub struct PersistRequest { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PersistSubrequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(string, optional, tag = "4")] + #[prost(string, optional, tag = "5")] pub follower_id: ::core::option::Option<::prost::alloc::string::String>, - #[prost(message, optional, tag = "5")] + #[prost(message, optional, tag = "6")] pub doc_batch: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -39,27 +41,31 @@ pub struct PersistResponse { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PersistSuccess { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(message, optional, tag = "4")] + #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PersistFailure { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(enumeration = "PersistFailureReason", tag = "4")] - pub failure_reason: i32, + #[prost(enumeration = "PersistFailureReason", tag = "5")] + pub reason: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -134,17 +140,19 @@ pub struct ReplicateRequest { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReplicateSubrequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(message, optional, tag = "4")] - pub from_position_exclusive: ::core::option::Option, #[prost(message, optional, tag = "5")] - pub to_position_inclusive: ::core::option::Option, + pub from_position_exclusive: ::core::option::Option, #[prost(message, optional, tag = "6")] + pub to_position_inclusive: ::core::option::Option, + #[prost(message, optional, tag = "7")] pub doc_batch: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -165,26 +173,30 @@ pub struct ReplicateResponse { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReplicateSuccess { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(message, optional, tag = "4")] + #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct ReplicateFailure { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, /// ingest.DocBatchV2 doc_batch = 4; /// ingest.IngestError error = 5; - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -266,7 +278,10 @@ pub struct PingResponse {} #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum PersistFailureReason { - ShardClosed = 0, + Unspecified = 0, + ShardClosed = 1, + RateLimited = 2, + ResourceExhausted = 3, } impl PersistFailureReason { /// String value of the enum field names used in the ProtoBuf definition. @@ -275,13 +290,21 @@ impl PersistFailureReason { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - PersistFailureReason::ShardClosed => "SHARD_CLOSED", + PersistFailureReason::Unspecified => "PERSIST_FAILURE_REASON_UNSPECIFIED", + PersistFailureReason::ShardClosed => "PERSIST_FAILURE_REASON_SHARD_CLOSED", + PersistFailureReason::RateLimited => "PERSIST_FAILURE_REASON_RATE_LIMITED", + PersistFailureReason::ResourceExhausted => { + "PERSIST_FAILURE_REASON_RESOURCE_EXHAUSTED" + } } } /// Creates an enum from field names used in the ProtoBuf definition. pub fn from_str_name(value: &str) -> ::core::option::Option { match value { - "SHARD_CLOSED" => Some(Self::ShardClosed), + "PERSIST_FAILURE_REASON_UNSPECIFIED" => Some(Self::Unspecified), + "PERSIST_FAILURE_REASON_SHARD_CLOSED" => Some(Self::ShardClosed), + "PERSIST_FAILURE_REASON_RATE_LIMITED" => Some(Self::RateLimited), + "PERSIST_FAILURE_REASON_RESOURCE_EXHAUSTED" => Some(Self::ResourceExhausted), _ => None, } } 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 c94574223dc..49865e6e2b3 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -11,11 +11,16 @@ pub struct IngestRequestV2 { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IngestSubrequest { - #[prost(string, tag = "1")] - pub index_id: ::prost::alloc::string::String, + /// The subrequest ID is used to identify the various subrequests and responses + /// (ingest, persist, replicate) at play during the ingest and replication + /// process. + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_id: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(message, optional, tag = "3")] + #[prost(message, optional, tag = "4")] pub doc_batch: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -23,42 +28,68 @@ pub struct IngestSubrequest { #[derive(Clone, PartialEq, ::prost::Message)] pub struct IngestResponseV2 { #[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 IngestSubresponse { - #[prost(string, tag = "1")] - pub index_id: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - #[prost(message, repeated, tag = "3")] pub successes: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "4")] + #[prost(message, repeated, tag = "2")] pub failures: ::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 IngestSuccess { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] + #[prost(uint64, tag = "4")] pub shard_id: u64, /// Replication position inclusive. - #[prost(message, optional, tag = "4")] + #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IngestFailure { - /// IngestFailureReason reason = 2; - #[prost(uint64, tag = "1")] - pub shard_id: u64, + #[prost(uint32, tag = "1")] + pub subrequest_id: u32, + #[prost(string, tag = "2")] + pub index_id: ::prost::alloc::string::String, + #[prost(string, tag = "3")] + pub source_id: ::prost::alloc::string::String, + #[prost(enumeration = "IngestFailureReason", tag = "5")] + pub reason: i32, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum IngestFailureReason { + Unspecified = 0, + NoShardsAvailable = 1, +} +impl IngestFailureReason { + /// 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 { + IngestFailureReason::Unspecified => "INGEST_FAILURE_REASON_UNSPECIFIED", + IngestFailureReason::NoShardsAvailable => { + "INGEST_FAILURE_REASON_NO_SHARDS_AVAILABLE" + } + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "INGEST_FAILURE_REASON_UNSPECIFIED" => Some(Self::Unspecified), + "INGEST_FAILURE_REASON_NO_SHARDS_AVAILABLE" => Some(Self::NoShardsAvailable), + _ => None, + } + } } /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index 0979e2a6ef3..e339aad99a0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -56,9 +56,10 @@ pub struct Shard { #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum CommitTypeV2 { - Auto = 0, - Wait = 1, - Force = 2, + Unspecified = 0, + Auto = 1, + Wait = 2, + Force = 3, } impl CommitTypeV2 { /// String value of the enum field names used in the ProtoBuf definition. @@ -67,17 +68,19 @@ impl CommitTypeV2 { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - CommitTypeV2::Auto => "AUTO", - CommitTypeV2::Wait => "WAIT", - CommitTypeV2::Force => "FORCE", + CommitTypeV2::Unspecified => "COMMIT_TYPE_V2_UNSPECIFIED", + CommitTypeV2::Auto => "COMMIT_TYPE_V2_AUTO", + CommitTypeV2::Wait => "COMMIT_TYPE_V2_WAIT", + CommitTypeV2::Force => "COMMIT_TYPE_V2_FORCE", } } /// Creates an enum from field names used in the ProtoBuf definition. pub fn from_str_name(value: &str) -> ::core::option::Option { match value { - "AUTO" => Some(Self::Auto), - "WAIT" => Some(Self::Wait), - "FORCE" => Some(Self::Force), + "COMMIT_TYPE_V2_UNSPECIFIED" => Some(Self::Unspecified), + "COMMIT_TYPE_V2_AUTO" => Some(Self::Auto), + "COMMIT_TYPE_V2_WAIT" => Some(Self::Wait), + "COMMIT_TYPE_V2_FORCE" => Some(Self::Force), _ => None, } } @@ -87,13 +90,14 @@ impl CommitTypeV2 { #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum ShardState { + Unspecified = 0, /// The shard is open and accepts write requests. - Open = 0, + Open = 1, /// The ingester hosting the shard is unavailable. - Unavailable = 1, + Unavailable = 2, /// The shard is closed and cannot be written to. /// It can be safely deleted if the publish position is superior or equal to the replication position. - Closed = 2, + Closed = 3, } impl ShardState { /// String value of the enum field names used in the ProtoBuf definition. @@ -102,17 +106,19 @@ impl ShardState { /// (if the ProtoBuf definition does not change) and safe for programmatic use. pub fn as_str_name(&self) -> &'static str { match self { - ShardState::Open => "OPEN", - ShardState::Unavailable => "UNAVAILABLE", - ShardState::Closed => "CLOSED", + ShardState::Unspecified => "SHARD_STATE_UNSPECIFIED", + ShardState::Open => "SHARD_STATE_OPEN", + ShardState::Unavailable => "SHARD_STATE_UNAVAILABLE", + ShardState::Closed => "SHARD_STATE_CLOSED", } } /// Creates an enum from field names used in the ProtoBuf definition. pub fn from_str_name(value: &str) -> ::core::option::Option { match value { - "OPEN" => Some(Self::Open), - "UNAVAILABLE" => Some(Self::Unavailable), - "CLOSED" => Some(Self::Closed), + "SHARD_STATE_UNSPECIFIED" => Some(Self::Unspecified), + "SHARD_STATE_OPEN" => Some(Self::Open), + "SHARD_STATE_UNAVAILABLE" => Some(Self::Unavailable), + "SHARD_STATE_CLOSED" => Some(Self::Closed), _ => None, } } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index dff55917add..9eb600aef13 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -276,7 +276,7 @@ pub struct OpenShardsSubresponse { #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, #[prost(message, repeated, tag = "3")] - pub open_shards: ::prost::alloc::vec::Vec, + pub opened_shards: ::prost::alloc::vec::Vec, #[prost(uint64, tag = "4")] pub next_shard_id: u64, } @@ -385,17 +385,18 @@ pub struct ListShardsSubresponse { #[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, + Unspecified = 0, + Cli = 1, + File = 2, + GcpPubsub = 3, + IngestV1 = 4, + IngestV2 = 5, + Kafka = 6, + Kinesis = 7, + Nats = 8, + Pulsar = 9, + Vec = 10, + Void = 11, } impl SourceType { /// String value of the enum field names used in the ProtoBuf definition. @@ -404,33 +405,35 @@ impl SourceType { /// (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", + SourceType::Unspecified => "SOURCE_TYPE_UNSPECIFIED", + SourceType::Cli => "SOURCE_TYPE_CLI", + SourceType::File => "SOURCE_TYPE_FILE", + SourceType::GcpPubsub => "SOURCE_TYPE_GCP_PUBSUB", + SourceType::IngestV1 => "SOURCE_TYPE_INGEST_V1", + SourceType::IngestV2 => "SOURCE_TYPE_INGEST_V2", + SourceType::Kafka => "SOURCE_TYPE_KAFKA", + SourceType::Kinesis => "SOURCE_TYPE_KINESIS", + SourceType::Nats => "SOURCE_TYPE_NATS", + SourceType::Pulsar => "SOURCE_TYPE_PULSAR", + SourceType::Vec => "SOURCE_TYPE_VEC", + SourceType::Void => "SOURCE_TYPE_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), + "SOURCE_TYPE_UNSPECIFIED" => Some(Self::Unspecified), + "SOURCE_TYPE_CLI" => Some(Self::Cli), + "SOURCE_TYPE_FILE" => Some(Self::File), + "SOURCE_TYPE_GCP_PUBSUB" => Some(Self::GcpPubsub), + "SOURCE_TYPE_INGEST_V1" => Some(Self::IngestV1), + "SOURCE_TYPE_INGEST_V2" => Some(Self::IngestV2), + "SOURCE_TYPE_KAFKA" => Some(Self::Kafka), + "SOURCE_TYPE_KINESIS" => Some(Self::Kinesis), + "SOURCE_TYPE_NATS" => Some(Self::Nats), + "SOURCE_TYPE_PULSAR" => Some(Self::Pulsar), + "SOURCE_TYPE_VEC" => Some(Self::Vec), + "SOURCE_TYPE_VOID" => Some(Self::Void), _ => None, } } diff --git a/quickwit/quickwit-proto/src/control_plane/mod.rs b/quickwit/quickwit-proto/src/control_plane/mod.rs index 793c344c3ea..e9f3168ba78 100644 --- a/quickwit/quickwit-proto/src/control_plane/mod.rs +++ b/quickwit/quickwit-proto/src/control_plane/mod.rs @@ -51,12 +51,11 @@ impl From for MetastoreError { } impl From for tonic::Status { - fn from(error: ControlPlaneError) -> Self { - let grpc_status_code = error.error_code().to_grpc_status_code(); - let error_json = - serde_json::to_string(&error).expect("control plane error should be JSON serializable"); - - tonic::Status::new(grpc_status_code, error_json) + fn from(control_plane_error: ControlPlaneError) -> Self { + let grpc_status_code = control_plane_error.error_code().to_grpc_status_code(); + let message_json = serde_json::to_string(&control_plane_error) + .unwrap_or_else(|_| format!("original control plane error: {control_plane_error}")); + tonic::Status::new(grpc_status_code, message_json) } } diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index ad590b73409..bbeaed67768 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -112,6 +112,9 @@ pub enum MetastoreError { #[error("access forbidden: {message}")] Forbidden { message: String }, + #[error("control plane state is inconsistent with that of the metastore")] + InconsistentControlPlaneState, + #[error("internal error: {message}; cause: `{cause}`")] Internal { message: String, cause: String }, @@ -160,10 +163,10 @@ impl From for MetastoreError { impl From for tonic::Status { fn from(metastore_error: MetastoreError) -> Self { - let grpc_code = metastore_error.error_code().to_grpc_status_code(); - let error_msg = serde_json::to_string(&metastore_error) - .unwrap_or_else(|_| format!("raw metastore error: {metastore_error}")); - tonic::Status::new(grpc_code, error_msg) + let grpc_status_code = metastore_error.error_code().to_grpc_status_code(); + let message_json = serde_json::to_string(&metastore_error) + .unwrap_or_else(|_| format!("original metastore error: {metastore_error}")); + tonic::Status::new(grpc_status_code, message_json) } } @@ -175,6 +178,7 @@ impl ServiceError for MetastoreError { Self::Db { .. } => ServiceErrorCode::Internal, Self::FailedPrecondition { .. } => ServiceErrorCode::BadRequest, Self::Forbidden { .. } => ServiceErrorCode::MethodNotAllowed, + Self::InconsistentControlPlaneState { .. } => ServiceErrorCode::BadRequest, Self::Internal { .. } => ServiceErrorCode::Internal, Self::InvalidArgument { .. } => ServiceErrorCode::BadRequest, Self::Io { .. } => ServiceErrorCode::Internal, @@ -210,6 +214,7 @@ impl SourceType { SourceType::Kinesis => "kinesis", SourceType::Nats => "nats", SourceType::Pulsar => "pulsar", + SourceType::Unspecified => "unspecified", SourceType::Vec => "vec", SourceType::Void => "void", } diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index 1a197f9bcd0..dcdc1b1b183 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -39,6 +39,8 @@ pub type SplitId = String; pub type ShardId = u64; +pub type SubrequestId = u32; + /// See the file `ingest.proto` for more details. pub type PublishToken = String; diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index b239a5e9a28..efe95991828 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -140,6 +140,7 @@ async fn ingest_v2( doc_lengths, }; let subrequest = IngestSubrequest { + subrequest_id: 0, index_id, source_id: INGEST_SOURCE_ID.to_string(), doc_batch: Some(doc_batch), From ee33492e77c0d4c1301a7e2f34fbf709159e6b9e Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 17:39:53 +0900 Subject: [PATCH 06/12] Bugfix (#4043) * Bugfix - Update the control plane model on toggle source. - Delete source from index config in control plane model on delete source. * Apply suggestions from code review Co-authored-by: Adrien Guillo --- .../src/control_plane.rs | 64 +++++++++++---- .../src/control_plane_model.rs | 79 ++++++++++++++++++- 2 files changed, 124 insertions(+), 19 deletions(-) diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index c6b3e80e089..6a2488a8e38 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -317,17 +317,21 @@ impl Handler for ControlPlane { request: ToggleSourceRequest, _ctx: &ActorContext, ) -> Result { + let index_uid: IndexUid = request.index_uid.clone().into(); + let source_id = request.source_id.clone(); + let enable = request.enable; + 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?; + let has_changed = self.model.toggle_source(&index_uid, &source_id, enable)?; - let response = EmptyResponse {}; - Ok(Ok(response)) + if has_changed { + self.indexing_scheduler.on_index_change(&self.model).await?; + } + + Ok(Ok(EmptyResponse {})) } } @@ -557,6 +561,7 @@ mod tests { #[tokio::test] async fn test_control_plane_toggle_source() { + quickwit_common::setup_logging_for_tests(); let universe = Universe::with_accelerated_time(); let cluster_id = "test-cluster".to_string(); @@ -565,20 +570,36 @@ mod tests { let ingester_pool = IngesterPool::default(); let mut mock_metastore = MetastoreServiceClient::mock(); + let mut index_metadata = IndexMetadata::for_test("test-index", "ram://toto"); + let test_source_config = SourceConfig::for_test("test-source", SourceParams::void()); + index_metadata.add_source(test_source_config).unwrap(); + mock_metastore + .expect_list_indexes_metadata() + .return_once(|_| { + Ok( + ListIndexesMetadataResponse::try_from_indexes_metadata(vec![index_metadata]) + .unwrap(), + ) + }); + mock_metastore .expect_toggle_source() - .withf(|toggle_source_request| { + .times(1) + .return_once(|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 {})); + Ok(EmptyResponse {}) + }); mock_metastore - .expect_list_indexes_metadata() - .returning(|_| { - Ok(ListIndexesMetadataResponse::try_from_indexes_metadata(Vec::new()).unwrap()) + .expect_toggle_source() + .times(1) + .return_once(|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); + Ok(EmptyResponse {}) }); + let replication_factor = 1; let (control_plane_mailbox, _control_plane_handle) = ControlPlane::spawn( @@ -590,17 +611,26 @@ mod tests { MetastoreServiceClient::from(mock_metastore), replication_factor, ); - let toggle_source_request = ToggleSourceRequest { + + let enabling_source_req = ToggleSourceRequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), enable: true, }; control_plane_mailbox - .ask_for_res(toggle_source_request) + .ask_for_res(enabling_source_req) .await .unwrap(); - // TODO: Test that delete index event is properly sent to ingest controller. + let disabling_source_req = ToggleSourceRequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + enable: false, + }; + control_plane_mailbox + .ask_for_res(disabling_source_req) + .await + .unwrap(); universe.assert_quit().await; } diff --git a/quickwit/quickwit-control-plane/src/control_plane_model.rs b/quickwit/quickwit-control-plane/src/control_plane_model.rs index 8fdb075ee9e..87c72abbf02 100644 --- a/quickwit/quickwit-control-plane/src/control_plane_model.rs +++ b/quickwit/quickwit-control-plane/src/control_plane_model.rs @@ -20,6 +20,7 @@ use std::collections::hash_map::Entry; use std::time::Instant; +use anyhow::bail; use fnv::{FnvHashMap, FnvHashSet}; #[cfg(test)] use itertools::Itertools; @@ -37,7 +38,7 @@ use quickwit_proto::metastore::{ }; use quickwit_proto::types::{IndexId, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; use serde::Serialize; -use tracing::{error, info}; +use tracing::{error, info, warn}; use crate::SourceUid; @@ -245,7 +246,35 @@ impl ControlPlaneModel { } pub(crate) fn delete_source(&mut self, index_uid: &IndexUid, source_id: &SourceId) { + // Removing shards from shard table. self.shard_table.delete_source(index_uid, source_id); + // Remove source from index config. + let Some(index_model) = self.index_table.get_mut(index_uid) else { + warn!(index_uid=%index_uid, source_id=%source_id, "delete source: index not found"); + return; + }; + if index_model.sources.remove(source_id).is_none() { + warn!(index_uid=%index_uid, source_id=%source_id, "delete source: source not found"); + }; + } + + /// Returns `true` if the source status has changed, `false` otherwise. + /// Returns an error if the source could not be found. + pub(crate) fn toggle_source( + &mut self, + index_uid: &IndexUid, + source_id: &SourceId, + enable: bool, + ) -> anyhow::Result { + let Some(index_model) = self.index_table.get_mut(index_uid) else { + bail!("index `{index_uid}` not found"); + }; + let Some(source_config) = index_model.sources.get_mut(source_id) else { + bail!("source `{source_id}` not found."); + }; + let has_changed = source_config.enabled != enable; + source_config.enabled = enable; + Ok(has_changed) } /// Removes the shards identified by their index UID, source ID, and shard IDs. @@ -492,7 +521,7 @@ impl ShardTable { #[cfg(test)] mod tests { - use quickwit_config::SourceConfig; + use quickwit_config::{SourceConfig, SourceParams}; use quickwit_metastore::IndexMetadata; use quickwit_proto::ingest::Shard; use quickwit_proto::metastore::ListIndexesMetadataResponse; @@ -859,4 +888,50 @@ mod tests { assert_eq!(shards.len(), 0); assert_eq!(table_entry.next_shard_id, 1); } + + #[test] + fn test_control_plane_model_toggle_source() { + let mut model = ControlPlaneModel::default(); + let index_metadata = IndexMetadata::for_test("test-index", "ram://"); + let index_uid = index_metadata.index_uid.clone(); + model.add_index(index_metadata); + let source_config = SourceConfig::for_test("test-source", SourceParams::void()); + model.add_source(&index_uid, source_config).unwrap(); + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), true) + .unwrap(); + assert!(!has_changed); + } + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), true) + .unwrap(); + assert!(!has_changed); + } + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), false) + .unwrap(); + assert!(has_changed); + } + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), false) + .unwrap(); + assert!(!has_changed); + } + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), true) + .unwrap(); + assert!(has_changed); + } + { + let has_changed = model + .toggle_source(&index_uid, &"test-source".to_string(), true) + .unwrap(); + assert!(!has_changed); + } + } } From ef44ac26278217f8aa3868de711623737d721ca1 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 17:47:46 +0900 Subject: [PATCH 07/12] Refactoring: Scheduling indexing plane does not return an error and is not async --- .../src/control_plane.rs | 28 +++++++-------- .../src/indexing_scheduler.rs | 35 ++++++------------- 2 files changed, 23 insertions(+), 40 deletions(-) diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 6a2488a8e38..fd099fa9c2e 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -126,13 +126,8 @@ impl Actor for ControlPlane { .await .context("failed to intialize the model")?; - if let Err(error) = self - .indexing_scheduler - .schedule_indexing_plan_if_needed(&self.model) - { - // TODO inspect error. - error!("error when scheduling indexing plan: `{}`.", error); - } + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); ctx.schedule_self_msg(CONTROL_PLAN_LOOP_INTERVAL, ControlPlanLoop) .await; @@ -149,9 +144,7 @@ impl Handler for ControlPlane { _message: ControlPlanLoop, ctx: &ActorContext, ) -> Result<(), ActorExitStatus> { - if let Err(error) = self.indexing_scheduler.control_running_plan(&self.model) { - error!("error when controlling the running plan: `{}`", error); - } + self.indexing_scheduler.control_running_plan(&self.model); ctx.schedule_self_msg(CONTROL_PLAN_LOOP_INTERVAL, ControlPlanLoop) .await; Ok(()) @@ -264,7 +257,8 @@ impl Handler for ControlPlane { // 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?; + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); Ok(Ok(response)) } @@ -299,7 +293,8 @@ impl Handler for ControlPlane { // 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?; + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); let response = EmptyResponse {}; Ok(Ok(response)) @@ -328,7 +323,8 @@ impl Handler for ControlPlane { let has_changed = self.model.toggle_source(&index_uid, &source_id, enable)?; if has_changed { - self.indexing_scheduler.on_index_change(&self.model).await?; + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); } Ok(Ok(EmptyResponse {})) @@ -352,7 +348,8 @@ impl Handler for ControlPlane { return convert_metastore_error(metastore_error); }; self.model.delete_source(&index_uid, &source_id); - self.indexing_scheduler.on_index_change(&self.model).await?; + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); let response = EmptyResponse {}; Ok(Ok(response)) } @@ -382,7 +379,8 @@ impl Handler for ControlPlane { } }; // TODO: Why do we return an error if the indexing scheduler fails? - self.indexing_scheduler.on_index_change(&self.model).await?; + self.indexing_scheduler + .schedule_indexing_plan_if_needed(&self.model); Ok(Ok(response)) } } diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs index 13bd1c8cabd..51d8ce6489a 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler.rs @@ -21,7 +21,6 @@ use std::cmp::Ordering; use std::fmt; use std::time::{Duration, Instant}; -use anyhow::Context; use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingService, IndexingTask}; @@ -134,14 +133,13 @@ impl IndexingScheduler { self.state.clone() } - pub(crate) fn schedule_indexing_plan_if_needed( - &mut self, - model: &ControlPlaneModel, - ) -> anyhow::Result<()> { + // Should be called whenever a change in the list of index/shard + // has happened. + pub(crate) fn schedule_indexing_plan_if_needed(&mut self, model: &ControlPlaneModel) { let mut indexers = self.get_indexers_from_indexer_pool(); if indexers.is_empty() { warn!("No indexer available, cannot schedule an indexing plan."); - return Ok(()); + return; }; let indexing_tasks = list_indexing_tasks(indexers.len(), model); let new_physical_plan = build_physical_indexing_plan(&indexers, indexing_tasks); @@ -152,19 +150,18 @@ impl IndexingScheduler { ); // No need to apply the new plan as it is the same as the old one. if plans_diff.is_empty() { - return Ok(()); + return; } } self.apply_physical_indexing_plan(&mut indexers, new_physical_plan); self.state.num_schedule_indexing_plan += 1; - Ok(()) } /// Checks if the last applied plan corresponds to the running indexing tasks present in the /// chitchat cluster state. If true, do nothing. /// - If node IDs differ, schedule a new indexing plan. /// - If indexing tasks differ, apply again the last plan. - pub(crate) fn control_running_plan(&mut self, model: &ControlPlaneModel) -> anyhow::Result<()> { + pub(crate) fn control_running_plan(&mut self, model: &ControlPlaneModel) { let last_applied_plan = if let Some(last_applied_plan) = self.state.last_applied_physical_plan.as_ref() { last_applied_plan @@ -172,15 +169,15 @@ impl IndexingScheduler { // If there is no plan, the node is probably starting and the scheduler did not find // indexers yet. In this case, we want to schedule as soon as possible to find new // indexers. - self.schedule_indexing_plan_if_needed(model)?; - return Ok(()); + self.schedule_indexing_plan_if_needed(model); + return; }; if let Some(last_applied_plan_timestamp) = self.state.last_applied_plan_timestamp { if Instant::now().duration_since(last_applied_plan_timestamp) < MIN_DURATION_BETWEEN_SCHEDULING { - return Ok(()); + return; } } @@ -196,13 +193,12 @@ impl IndexingScheduler { ); if !indexing_plans_diff.has_same_nodes() { info!(plans_diff=?indexing_plans_diff, "Running plan and last applied plan node IDs differ: schedule an indexing plan."); - self.schedule_indexing_plan_if_needed(model)?; + self.schedule_indexing_plan_if_needed(model); } else if !indexing_plans_diff.has_same_tasks() { // Some nodes may have not received their tasks, apply it again. info!(plans_diff=?indexing_plans_diff, "Running tasks and last applied tasks differ: reapply last plan."); self.apply_physical_indexing_plan(&mut indexers, last_applied_plan.clone()); } - Ok(()) } fn get_indexers_from_indexer_pool(&self) -> Vec<(String, IndexerNodeInfo)> { @@ -243,17 +239,6 @@ impl IndexingScheduler { self.state.last_applied_plan_timestamp = Some(Instant::now()); self.state.last_applied_physical_plan = Some(new_physical_plan); } - - // Should be called whenever a change in the list of index/shard - // has happened - pub(crate) async fn on_index_change( - &mut self, - model: &ControlPlaneModel, - ) -> anyhow::Result<()> { - self.schedule_indexing_plan_if_needed(model) - .context("error when scheduling indexing plan")?; - Ok(()) - } } struct IndexingPlansDiff<'a> { From a0d5475a87ec5f203e89867fc497ce9fe79c0de4 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 17:58:01 +0900 Subject: [PATCH 08/12] Removing ambiguity between the different ways to create an IndexUid. (#4038) The code will now panic if the arguments are wrong. Ideally we would prefer to have a TryFrom implementation. Before this PR, some client argument could silently create an IndexUi thatwere breaking a contract (IndexUid wrapping a string without a ":k"). After this PR, some client argument could create a panic. This partly addresses to #3943. --- .../src/control_plane.rs | 2 +- .../src/indexing_plan.rs | 2 +- .../src/garbage_collection.rs | 8 +- quickwit/quickwit-indexing/failpoints/mod.rs | 2 +- .../quickwit-indexing/src/actors/indexer.rs | 22 +++--- .../src/actors/indexing_pipeline.rs | 2 +- .../src/actors/merge_pipeline.rs | 2 +- .../src/actors/merge_planner.rs | 16 ++-- .../quickwit-indexing/src/actors/packager.rs | 2 +- .../quickwit-indexing/src/actors/publisher.rs | 2 +- .../quickwit-indexing/src/actors/uploader.rs | 8 +- .../quickwit-indexing/src/merge_policy/mod.rs | 6 +- .../src/source/file_source.rs | 6 +- .../src/source/gcp_pubsub_source.rs | 4 +- .../src/source/ingest_api_source.rs | 10 +-- .../src/source/kafka_source.rs | 6 +- .../src/source/pulsar_source.rs | 2 +- .../src/source/source_factory.rs | 2 +- .../src/source/vec_source.rs | 4 +- .../src/source/void_source.rs | 4 +- quickwit/quickwit-indexing/src/test_utils.rs | 5 +- .../file_backed_index/mod.rs | 43 +++++------ .../metastore/file_backed_metastore/mod.rs | 18 ++--- .../src/metastore/index_metadata/mod.rs | 4 +- .../src/metastore/index_metadata/serialize.rs | 2 +- .../src/metastore/postgresql_metastore.rs | 6 +- quickwit/quickwit-metastore/src/tests/mod.rs | 77 +++++++++++-------- .../file-backed-index/v0.4.expected.json | 6 +- .../file-backed-index/v0.5.expected.json | 6 +- .../index-metadata/v0.4.expected.json | 6 +- .../index-metadata/v0.5.expected.json | 6 +- .../split-metadata/v0.4.expected.json | 2 +- .../split-metadata/v0.5.expected.json | 2 +- quickwit/quickwit-proto/src/types/mod.rs | 46 +++++++---- 34 files changed, 187 insertions(+), 154 deletions(-) diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index fd099fa9c2e..05a8c554232 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -785,7 +785,7 @@ mod tests { Ok(list_shards_resp) }, ); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let index_uid_string = index_uid.to_string(); mock_metastore.expect_create_index().times(1).return_once( |_create_index_request: CreateIndexRequest| { diff --git a/quickwit/quickwit-control-plane/src/indexing_plan.rs b/quickwit/quickwit-control-plane/src/indexing_plan.rs index c3f1d14e8e5..f987420f30c 100644 --- a/quickwit/quickwit-control-plane/src/indexing_plan.rs +++ b/quickwit/quickwit-control-plane/src/indexing_plan.rs @@ -720,7 +720,7 @@ mod tests { prop_compose! { fn gen_kafka_source() (index_idx in 0usize..100usize, desired_num_pipelines in 1usize..51usize, max_num_pipelines_per_indexer in 1usize..5usize) -> (IndexUid, SourceConfig) { - let index_uid = IndexUid::from_parts(format!("index-id-{index_idx}"), "" /* this is the index uid */); + let index_uid = IndexUid::from_parts(&format!("index-id-{index_idx}"), "" /* this is the index uid */); let source_id = append_random_suffix("kafka-source"); (index_uid, SourceConfig { source_id, diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index b8c0633403e..d0c6d61a097 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -475,7 +475,7 @@ mod tests { let split_id = "test-run-gc--split"; let split_metadata = SplitMetadata { split_id: split_id.to_string(), - index_uid: IndexUid::new(index_id), + index_uid: IndexUid::new_with_random_ulid(index_id), ..Default::default() }; let stage_splits_request = @@ -566,7 +566,7 @@ mod tests { .times(2) .returning(|_| Ok(ListSplitsResponse::empty())); run_garbage_collect( - IndexUid::new("index-test-gc-deletes"), + IndexUid::new_with_random_ulid("index-test-gc-deletes"), storage.clone(), MetastoreServiceClient::from(metastore), Duration::from_secs(30), @@ -597,7 +597,7 @@ mod tests { let split_id = "test-delete-splits-happy--split"; let split_metadata = SplitMetadata { split_id: split_id.to_string(), - index_uid: IndexUid::new(index_id), + index_uid: IndexUid::new_with_random_ulid(index_id), ..Default::default() }; let stage_splits_request = @@ -772,7 +772,7 @@ mod tests { let storage = Arc::new(mock_storage); let index_id = "test-delete-splits-storage-error--index"; - let index_uid = IndexUid::new(index_id.to_string()); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore.expect_delete_splits().return_once(|_| { diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index c36958dc128..23e211b413e 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -297,7 +297,7 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul tantivy_dirs, }; let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new(index_id.to_string()), + index_uid: IndexUid::new_with_random_ulid(index_id.to_string()), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index cd19c98e938..85bc7907428 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -715,7 +715,7 @@ mod tests { #[tokio::test] async fn test_indexer_triggers_commit_on_target_num_docs() -> anyhow::Result<()> { - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -852,7 +852,7 @@ 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 index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -931,7 +931,7 @@ mod tests { async fn test_indexer_triggers_commit_on_timeout() -> anyhow::Result<()> { let universe = Universe::new(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1015,7 +1015,7 @@ mod tests { async fn test_indexer_triggers_commit_on_drained_mailbox() -> anyhow::Result<()> { let universe = Universe::new(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1090,7 +1090,7 @@ mod tests { async fn test_indexer_triggers_commit_on_quit() -> anyhow::Result<()> { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1169,7 +1169,7 @@ mod tests { async fn test_indexer_partitioning() -> anyhow::Result<()> { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1267,7 +1267,7 @@ mod tests { async fn test_indexer_exceeding_max_num_partitions() { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1337,7 +1337,7 @@ mod tests { async fn test_indexer_propagates_publish_lock() { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1409,7 +1409,7 @@ mod tests { async fn test_indexer_ignores_messages_when_publish_lock_is_dead() { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1474,7 +1474,7 @@ mod tests { async fn test_indexer_honors_batch_commit_request() { let universe = Universe::with_accelerated_time(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -1535,7 +1535,7 @@ mod tests { #[tokio::test] async fn test_indexer_checkpoint_on_all_failed_docs() -> anyhow::Result<()> { let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 5e9b9864888..ad7bb4eedca 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -833,7 +833,7 @@ mod tests { let node_id = "test-node"; let doc_mapper = Arc::new(default_doc_mapper_for_test()); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: node_id.to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 7c1314bd199..1a094c579cb 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -499,7 +499,7 @@ mod tests { #[tokio::test] async fn test_merge_pipeline_simple() -> anyhow::Result<()> { let mut metastore = MetastoreServiceClient::mock(); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), diff --git a/quickwit/quickwit-indexing/src/actors/merge_planner.rs b/quickwit/quickwit-indexing/src/actors/merge_planner.rs index 99a2f18ec98..50157d9fc36 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_planner.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_planner.rs @@ -506,7 +506,7 @@ mod tests { #[tokio::test] async fn test_merge_planner_with_stable_custom_merge_policy() -> anyhow::Result<()> { let universe = Universe::with_accelerated_time(); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe.create_test_mailbox(); let pipeline_id = IndexingPipelineId { @@ -594,7 +594,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe.create_test_mailbox(); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -646,7 +646,7 @@ mod tests { let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -745,7 +745,7 @@ mod tests { let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -823,7 +823,7 @@ mod tests { let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid, source_id: "test-source".to_string(), @@ -842,7 +842,7 @@ mod tests { }; // It is different from the index_uid because the index uid has a unique suffix. - let other_index_uid = IndexUid::new("test-index"); + let other_index_uid = IndexUid::new_with_random_ulid("test-index"); let pre_existing_splits = vec![ split_metadata_for_test( @@ -887,7 +887,7 @@ mod tests { let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), @@ -967,7 +967,7 @@ mod tests { let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Unbounded); - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), diff --git a/quickwit/quickwit-indexing/src/actors/packager.rs b/quickwit/quickwit-indexing/src/actors/packager.rs index 200aad9253b..7ba54b82b08 100644 --- a/quickwit/quickwit-indexing/src/actors/packager.rs +++ b/quickwit/quickwit-indexing/src/actors/packager.rs @@ -404,7 +404,7 @@ mod tests { } let index = index_writer.finalize()?; let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 4c5a769dfcd..aad20a9b665 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -419,7 +419,7 @@ mod tests { publisher_mailbox .send_message(SplitsUpdate { - index_uid: IndexUid::new("index"), + index_uid: IndexUid::new_with_random_ulid("index"), new_splits: vec![SplitMetadata::for_test("test-split".to_string())], replaced_split_ids: Vec::new(), checkpoint_delta_opt: None, diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index ecd78847d94..cc4588757dd 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -504,7 +504,7 @@ mod tests { let event_broker = EventBroker::default(); let universe = Universe::new(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -615,7 +615,7 @@ mod tests { #[tokio::test] async fn test_uploader_with_sequencer_emits_replace() -> anyhow::Result<()> { let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -867,7 +867,7 @@ mod tests { }; uploader_mailbox .send_message(EmptySplit { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), checkpoint_delta, publish_lock: PublishLock::default(), publish_token_opt: None, @@ -943,7 +943,7 @@ mod tests { let universe = Universe::new(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-indexing/src/merge_policy/mod.rs b/quickwit/quickwit-indexing/src/merge_policy/mod.rs index 9b1cfe34c29..d264f8a9f20 100644 --- a/quickwit/quickwit-indexing/src/merge_policy/mod.rs +++ b/quickwit/quickwit-indexing/src/merge_policy/mod.rs @@ -332,7 +332,7 @@ pub mod tests { let merged_split_id = new_split_id(); let tags = merge_tags(splits); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test_index"), + index_uid: IndexUid::new_with_random_ulid("test_index"), source_id: "test_source".to_string(), node_id: "test_node".to_string(), pipeline_ord: 0, @@ -363,7 +363,7 @@ pub mod tests { let (merge_op_mailbox, merge_op_inbox) = universe.create_test_mailbox::(); let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, @@ -424,7 +424,7 @@ pub mod tests { maturity, tags: BTreeSet::from_iter(vec!["tenant_id:1".to_string(), "tenant_id:2".to_string()]), footer_offsets: 0..100, - index_uid: IndexUid::new("test-index"), + index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), node_id: "test-node".to_string(), ..Default::default() diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index 265e2f0e3d3..40e57f163c3 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -224,7 +224,7 @@ mod tests { let metastore = metastore_for_test(); let file_source = FileSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), @@ -291,7 +291,7 @@ mod tests { let metastore = metastore_for_test(); let source = FileSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), @@ -382,7 +382,7 @@ mod tests { let metastore = metastore_for_test(); let source = FileSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), diff --git a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs index 4dc332d4e6b..389ec96d366 100644 --- a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs +++ b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs @@ -354,7 +354,7 @@ mod gcp_pubsub_emulator_tests { let source_config = get_source_config(&subscription); let index_id = append_random_suffix("test-gcp-pubsub-source--invalid-subscription--index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let metastore = metastore_for_test(); let SourceParams::GcpPubSub(params) = source_config.clone().source_params else { panic!( @@ -385,7 +385,7 @@ mod gcp_pubsub_emulator_tests { let source_loader = quickwit_supported_sources(); let metastore = metastore_for_test(); let index_id: String = append_random_suffix("test-gcp-pubsub-source--index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let mut pubsub_messages = Vec::with_capacity(6); for i in 0..6 { diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index 54ca2c4ceee..54d7ce7813b 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -272,7 +272,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let metastore = metastore_for_test(); let index_id = append_random_suffix("test-ingest-api-source"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let temp_dir = tempfile::tempdir()?; let queues_dir_path = temp_dir.path(); @@ -361,7 +361,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let metastore = metastore_for_test(); let index_id = append_random_suffix("test-ingest-api-source"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let temp_dir = tempfile::tempdir()?; let queues_dir_path = temp_dir.path(); let ingest_api_service = @@ -431,7 +431,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let metastore = metastore_for_test(); let index_id = append_random_suffix("test-ingest-api-source"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let temp_dir = tempfile::tempdir()?; let queues_dir_path = temp_dir.path(); let ingest_api_service = @@ -485,7 +485,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let metastore = metastore_for_test(); let index_id = append_random_suffix("test-ingest-api-source"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let temp_dir = tempfile::tempdir()?; let queues_dir_path = temp_dir.path(); @@ -552,7 +552,7 @@ mod tests { let universe = Universe::with_accelerated_time(); let metastore = metastore_for_test(); let index_id = append_random_suffix("test-ingest-api-source"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let temp_dir = tempfile::tempdir()?; let queues_dir_path = temp_dir.path(); diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index 908a488cba8..d2db3477425 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -1002,7 +1002,7 @@ mod kafka_broker_tests { let metastore = metastore_for_test(); let index_id = append_random_suffix("test-kafka-source--process-message--index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let (_source_id, source_config) = get_source_config(&topic); let SourceParams::Kafka(params) = source_config.clone().source_params else { panic!( @@ -1191,7 +1191,7 @@ mod kafka_broker_tests { let metastore = metastore_for_test(); let index_id = append_random_suffix("test-kafka-source--process-revoke--partitions--index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let (_source_id, source_config) = get_source_config(&topic); let SourceParams::Kafka(params) = source_config.clone().source_params else { panic!( @@ -1249,7 +1249,7 @@ mod kafka_broker_tests { let metastore = metastore_for_test(); let index_id = append_random_suffix("test-kafka-source--process-partition-eof--index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let (_source_id, source_config) = get_source_config(&topic); let SourceParams::Kafka(params) = source_config.clone().source_params else { panic!( diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index 50b91b12928..691609464ce 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -815,7 +815,7 @@ mod pulsar_broker_tests { let topic = append_random_suffix("test-pulsar-source-topic"); let index_id = append_random_suffix("test-pulsar-source-index"); - let index_uid = IndexUid::new(&index_id); + let index_uid = IndexUid::new_with_random_ulid(&index_id); let (_source_id, source_config) = get_source_config([&topic]); let params = if let SourceParams::Pulsar(params) = source_config.clone().source_params { params diff --git a/quickwit/quickwit-indexing/src/source/source_factory.rs b/quickwit/quickwit-indexing/src/source/source_factory.rs index 72aa013df9c..28bff181ca3 100644 --- a/quickwit/quickwit-indexing/src/source/source_factory.rs +++ b/quickwit/quickwit-indexing/src/source/source_factory.rs @@ -144,7 +144,7 @@ mod tests { source_loader .load_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index f23fe8a0bbe..79271a53712 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -168,7 +168,7 @@ mod tests { let metastore = metastore_for_test(); let vec_source = VecSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), @@ -229,7 +229,7 @@ mod tests { let metastore = metastore_for_test(); let vec_source = VecSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), diff --git a/quickwit/quickwit-indexing/src/source/void_source.rs b/quickwit/quickwit-indexing/src/source/void_source.rs index c4d204ea3d5..113133e09a9 100644 --- a/quickwit/quickwit-indexing/src/source/void_source.rs +++ b/quickwit/quickwit-indexing/src/source/void_source.rs @@ -97,7 +97,7 @@ mod tests { }; let metastore = metastore_for_test(); let ctx = SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), @@ -124,7 +124,7 @@ mod tests { let metastore = metastore_for_test(); let void_source = VoidSourceFactory::typed_create_source( SourceRuntimeArgs::for_test( - IndexUid::new("test-index"), + IndexUid::new_with_random_ulid("test-index"), source_config, metastore, PathBuf::from("./queues"), diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 8327588cfd7..050a062cc29 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -237,7 +237,10 @@ pub struct MockSplitBuilder { impl MockSplitBuilder { pub fn new(split_id: &str) -> Self { Self { - split_metadata: mock_split_meta(split_id, &IndexUid::new("test-index")), + split_metadata: mock_split_meta( + split_id, + &IndexUid::from_parts("test-index", "000000"), + ), } } 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 0d83315aca9..29edd6f1ecd 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 @@ -783,55 +783,54 @@ mod tests { fn test_single_filter_behaviour() { let [split_1, split_2, split_3] = make_splits(); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_split_state(SplitState::Staged); assert!(split_query_predicate(&&split_1, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_split_state(SplitState::Published); assert!(!split_query_predicate(&&split_2, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_split_states([SplitState::Published, SplitState::MarkedForDeletion]); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = - ListSplitsQuery::for_index(IndexUid::new("test-index")).with_update_timestamp_lt(51); + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_update_timestamp_lt(51); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = - ListSplitsQuery::for_index(IndexUid::new("test-index")).with_create_timestamp_gte(51); + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_create_timestamp_gte(51); assert!(!split_query_predicate(&&split_1, &query)); assert!(!split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = - ListSplitsQuery::for_index(IndexUid::new("test-index")).with_delete_opstamp_gte(4); + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_delete_opstamp_gte(4); assert!(split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(!split_query_predicate(&&split_3, &query)); - let query = - ListSplitsQuery::for_index(IndexUid::new("test-index")).with_time_range_start_gt(45); + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_time_range_start_gt(45); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = - ListSplitsQuery::for_index(IndexUid::new("test-index")).with_time_range_end_lt(45); + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_time_range_end_lt(45); assert!(split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")).with_tags_filter( - TagFilterAst::Tag { + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) + .with_tags_filter(TagFilterAst::Tag { is_present: false, tag: "tag-2".to_string(), - }, - ); + }); assert!(split_query_predicate(&&split_1, &query)); assert!(!split_query_predicate(&&split_2, &query)); assert!(!split_query_predicate(&&split_3, &query)); @@ -841,35 +840,35 @@ mod tests { fn test_combination_filter() { let [split_1, split_2, split_3] = make_splits(); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_time_range_start_gt(0) .with_time_range_end_lt(40); assert!(split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_time_range_start_gt(45) .with_delete_opstamp_gt(0); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(!split_query_predicate(&&split_3, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_update_timestamp_lt(51) .with_split_states([SplitState::Published, SplitState::MarkedForDeletion]); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(split_query_predicate(&&split_3, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_update_timestamp_lt(51) .with_create_timestamp_lte(63); assert!(!split_query_predicate(&&split_1, &query)); assert!(split_query_predicate(&&split_2, &query)); assert!(!split_query_predicate(&&split_3, &query)); - let query = ListSplitsQuery::for_index(IndexUid::new("test-index")) + let query = ListSplitsQuery::for_index(IndexUid::new_with_random_ulid("test-index")) .with_time_range_start_gt(90) .with_tags_filter(TagFilterAst::Tag { is_present: true, diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs index 78d9b7b9cb5..b4aa4b446ab 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -1057,14 +1057,14 @@ mod tests { // Open a non-existent index. let metastore_error = metastore - .get_index(IndexUid::new("index-does-not-exist")) + .get_index(IndexUid::new_with_random_ulid("index-does-not-exist")) .await .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::NotFound { .. })); // Open a index with a different incarnation_id. let metastore_error = metastore - .get_index(IndexUid::new(index_id)) + .get_index(IndexUid::new_with_random_ulid(index_id)) .await .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::NotFound { .. })); @@ -1183,7 +1183,7 @@ mod tests { // Getting index with inconsistent index ID should raise an error. let metastore_error = metastore - .get_index(IndexUid::new(index_id)) + .get_index(IndexUid::new_with_random_ulid(index_id)) .await .unwrap_err(); assert!(matches!(metastore_error, MetastoreError::Internal { .. })); @@ -1369,7 +1369,7 @@ mod tests { 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_uid = IndexUid::new_with_random_ulid(&format!("test-index-{idx}")); let index_config = IndexConfig::for_test(index_uid.index_id(), "ram:///indexes/test-index"); let create_index_request = @@ -1446,7 +1446,7 @@ mod tests { assert!(matches!(metastore_error, MetastoreError::Internal { .. })); // Try fetch the not created index. let created_index_error = metastore - .get_index(IndexUid::new(index_id)) + .get_index(IndexUid::new_with_random_ulid(index_id)) .await .unwrap_err(); assert!(matches!( @@ -1462,7 +1462,7 @@ mod tests { let ram_storage_clone = ram_storage.clone(); let ram_storage_clone_2 = ram_storage.clone(); let index_id = "test-index"; - let index_uid = IndexUid::new(index_id); + let index_uid = IndexUid::new_with_random_ulid(index_id); mock_storage // remove this if we end up changing the semantics of create. .expect_exists() @@ -1571,7 +1571,7 @@ mod tests { // Let's fetch the index, we expect an internal error as the index state is in `Creating` // state. let created_index_error = metastore - .get_index(IndexUid::new(index_id)) + .get_index(IndexUid::new_with_random_ulid(index_id)) .await .unwrap_err(); assert!(matches!( @@ -1586,7 +1586,7 @@ mod tests { let ram_storage = RamStorage::default(); let ram_storage_clone = ram_storage.clone(); let index_id = "test-index"; - let index_uid = IndexUid::new(index_id); + let index_uid = IndexUid::new_with_random_ulid(index_id); let index_metadata = IndexMetadata::for_test(index_uid.index_id(), "ram:///indexes/test-index"); let index = FileBackedIndex::from(index_metadata); @@ -1633,7 +1633,7 @@ mod tests { let ram_storage = RamStorage::default(); let ram_storage_clone = ram_storage.clone(); let index_id = "test-index"; - let index_uid = IndexUid::new(index_id); + let index_uid = IndexUid::new_with_random_ulid(index_id); let index_metadata = IndexMetadata::for_test(index_uid.index_id(), "ram:///indexes/test-index"); let index = FileBackedIndex::from(index_metadata); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index 050a9317893..24129bec968 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -53,7 +53,7 @@ pub struct IndexMetadata { impl IndexMetadata { /// Panics if `index_config` is missing `index_uri`. pub fn new(index_config: IndexConfig) -> Self { - let index_uid = IndexUid::new(index_config.index_id.clone()); + let index_uid = IndexUid::new_with_random_ulid(&index_config.index_id); IndexMetadata::new_with_index_uid(index_uid, index_config) } @@ -154,7 +154,7 @@ impl TestableForRegression for IndexMetadata { let checkpoint = IndexCheckpoint::from(per_source_checkpoint); let index_config = IndexConfig::sample_for_regression(); let mut index_metadata = IndexMetadata { - index_uid: IndexUid::from_parts(index_config.index_id.clone(), Ulid::nil()), + index_uid: IndexUid::from_parts(&index_config.index_id, Ulid::nil()), index_config, checkpoint, create_timestamp: 1789, diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs index 9dd810a99da..00adaede029 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/serialize.rs @@ -97,7 +97,7 @@ impl TryFrom for IndexMetadata { } Ok(Self { index_uid: if v0_6.index_uid.is_empty() { - v0_6.index_config.index_id.clone().into() + IndexUid::from_parts(&v0_6.index_config.index_id, "") } else { v0_6.index_uid }, diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index 5f9648296e1..c1fba530a80 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -1596,7 +1596,7 @@ mod tests { #[test] fn test_single_sql_query_builder() { - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); let sql = build_query_filter(String::new(), &query); @@ -1699,7 +1699,7 @@ mod tests { #[test] fn test_combination_sql_query_builder() { - let index_uid = IndexUid::new("test-index"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let query = ListSplitsQuery::for_index(index_uid.clone()) .with_time_range_start_gt(0) .with_time_range_end_lt(40); @@ -1751,7 +1751,7 @@ mod tests { ) ); - let index_uid_2 = IndexUid::new("test-index-2"); + let index_uid_2 = IndexUid::new_with_random_ulid("test-index-2"); let query = ListSplitsQuery::try_from_index_uids(vec![index_uid.clone(), index_uid_2.clone()]) .unwrap(); diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index 21a456014f2..88101809739 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -387,9 +387,10 @@ pub async fn test_metastore_delete_index< let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); + let index_uid_not_existing = IndexUid::new_with_random_ulid("index-not-found"); let error = metastore .delete_index(DeleteIndexRequest { - index_uid: "index-not-found".to_string(), + index_uid: index_uid_not_existing.to_string(), }) .await .unwrap_err(); @@ -400,7 +401,7 @@ pub async fn test_metastore_delete_index< let error = metastore .delete_index(DeleteIndexRequest { - index_uid: "test-delete-index".to_string(), + index_uid: index_uid_not_existing.to_string(), }) .await .unwrap_err(); @@ -534,7 +535,7 @@ pub async fn test_metastore_add_source Option<(IndexUid, SourceId, ShardId)> { /// Index identifiers that uniquely identify not only the index, but also /// its incarnation allowing to distinguish between deleted and recreated indexes. -/// It is represented as a stiring in index_id:incarnation_id format. -#[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Eq, Ord, PartialOrd, Hash)] +/// It is represented as a string in index_id:incarnation_id format. +#[derive(Clone, Debug, Default, Serialize, PartialEq, Eq, Ord, PartialOrd, Hash)] pub struct IndexUid(String); +// It is super lame, but for backward compatibility reasons we accept having a missing ulid part. +// TODO DEPRECATED ME and remove +impl<'de> Deserialize<'de> for IndexUid { + fn deserialize(deserializer: D) -> Result + where D: Deserializer<'de> { + let index_uid_str: String = String::deserialize(deserializer)?; + if !index_uid_str.contains(':') { + return Ok(IndexUid::from_parts(&index_uid_str, "")); + } + let index_uid = IndexUid::from(index_uid_str); + Ok(index_uid) + } +} + impl fmt::Display for IndexUid { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { write!(f, "{}", self.0) @@ -73,8 +87,8 @@ impl fmt::Display for IndexUid { impl IndexUid { /// Creates a new index uid from index_id. - /// A random UUID will be used as incarnation - pub fn new(index_id: impl Into) -> Self { + /// A random ULID will be used as incarnation + pub fn new_with_random_ulid(index_id: &str) -> Self { Self::from_parts(index_id, Ulid::new().to_string()) } @@ -87,14 +101,9 @@ impl IndexUid { &self.0 } - pub fn from_parts(index_id: impl Into, incarnation_id: impl Into) -> Self { - let incarnation_id = incarnation_id.into(); - let index_id = index_id.into(); - if incarnation_id.is_empty() { - Self(index_id) - } else { - Self(format!("{index_id}:{incarnation_id}")) - } + pub fn from_parts(index_id: &str, incarnation_id: impl Display) -> Self { + assert!(!index_id.contains(':'), "Index id may not contain `:`"); + Self(format!("{index_id}:{incarnation_id}")) } pub fn index_id(&self) -> &str { @@ -127,8 +136,15 @@ impl From<&str> for IndexUid { } impl From for IndexUid { - fn from(index_uid: String) -> Self { - Self(index_uid) + fn from(index_uid: String) -> IndexUid { + let count_colon = index_uid + .as_bytes() + .iter() + .copied() + .filter(|c| *c == b':') + .count(); + assert_eq!(count_colon, 1, "Invalid index_uid: {}", index_uid); + IndexUid(index_uid) } } From a83f845c45bb22aec63ed76e580cee26c074559d Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 18:01:24 +0900 Subject: [PATCH 09/12] Clippy (#4045) --- quickwit/quickwit-index-management/src/garbage_collection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index d0c6d61a097..f0062ff8921 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -772,7 +772,7 @@ mod tests { let storage = Arc::new(mock_storage); let index_id = "test-delete-splits-storage-error--index"; - let index_uid = IndexUid::new_with_random_ulid(&index_id); + let index_uid = IndexUid::new_with_random_ulid(index_id); let mut mock_metastore = MetastoreServiceClient::mock(); mock_metastore.expect_delete_splits().return_once(|_| { From 98a08ffd0fb20fa9c78c3cb1954e875d549a9c05 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Sat, 28 Oct 2023 05:39:06 -0400 Subject: [PATCH 10/12] Fix tests --- quickwit/quickwit-indexing/failpoints/mod.rs | 2 +- quickwit/quickwit-proto/src/types/mod.rs | 4 ++-- quickwit/quickwit-search/src/root.rs | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index 23e211b413e..95254a5461a 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -297,7 +297,7 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul tantivy_dirs, }; let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new_with_random_ulid(index_id.to_string()), + index_uid: IndexUid::new_with_random_ulid(index_id), source_id: "test-source".to_string(), node_id: "test-node".to_string(), pipeline_ord: 0, diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index 54fc7ffcd02..1019b8ca7b6 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -102,7 +102,7 @@ impl IndexUid { } pub fn from_parts(index_id: &str, incarnation_id: impl Display) -> Self { - assert!(!index_id.contains(':'), "Index id may not contain `:`"); + assert!(!index_id.contains(':'), "index ID may not contain `:`"); Self(format!("{index_id}:{incarnation_id}")) } @@ -143,7 +143,7 @@ impl From for IndexUid { .copied() .filter(|c| *c == b':') .count(); - assert_eq!(count_colon, 1, "Invalid index_uid: {}", index_uid); + assert_eq!(count_colon, 1, "invalid index UID: `{}`", index_uid); IndexUid(index_uid) } } diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 367a3416400..16d0264e192 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -81,7 +81,7 @@ impl SearchJob { #[cfg(test)] pub fn for_test(split_id: &str, cost: usize) -> SearchJob { SearchJob { - index_uid: IndexUid::from("test-index".to_string()), + index_uid: IndexUid::from("test-index:0"), cost, offsets: SplitIdAndFooterOffsets { split_id: split_id.to_string(), From 96f5504339cb3ce941b10e3db144f9af14b2cc7c Mon Sep 17 00:00:00 2001 From: Kamalesh Palanisamy Date: Sat, 28 Oct 2023 08:58:36 -0500 Subject: [PATCH 11/12] Update sql strings with sea_query sql statements (#3959) --- quickwit/Cargo.lock | 46 ++ quickwit/Cargo.toml | 2 + quickwit/quickwit-metastore/Cargo.toml | 2 + .../src/metastore/postgresql_metastore.rs | 443 +++++++++++------- .../src/metastore/postgresql_model.rs | 26 + 5 files changed, 341 insertions(+), 178 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 9a7ae05bb2b..a3902f31d5d 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -3020,6 +3020,17 @@ version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "64e9829a50b42bb782c1df523f78d332fe371b10c661e78b7a3c34b0198e9fac" +[[package]] +name = "inherent" +version = "1.0.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ce243b1bfa62ffc028f1cc3b6034ec63d649f3031bc8a4fbbb004e1ac17d1f68" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.38", +] + [[package]] name = "inout" version = "0.1.3" @@ -5528,6 +5539,8 @@ dependencies = [ "quickwit-storage", "rand 0.8.5", "regex", + "sea-query", + "sea-query-binder", "serde", "serde_json", "serde_with 3.4.0", @@ -6485,6 +6498,39 @@ dependencies = [ "untrusted 0.9.0", ] +[[package]] +name = "sea-query" +version = "0.30.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fb3e6bba153bb198646c8762c48414942a38db27d142e44735a133cabddcc820" +dependencies = [ + "inherent", + "sea-query-derive", +] + +[[package]] +name = "sea-query-binder" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "36bbb68df92e820e4d5aeb17b4acd5cc8b5d18b2c36a4dd6f4626aabfa7ab1b9" +dependencies = [ + "sea-query", + "sqlx", +] + +[[package]] +name = "sea-query-derive" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bd78f2e0ee8e537e9195d1049b752e0433e2cac125426bccb7b5c3e508096117" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "syn 1.0.109", + "thiserror", +] + [[package]] name = "seahash" version = "4.1.0" diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 261add432f8..e8f45a7bf5a 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -142,6 +142,8 @@ reqwest = { version = "0.11", default-features = false, features = [ "rustls-tls", ] } rust-embed = "6.8.1" +sea-query = { version = "0" } +sea-query-binder = { version = "0", features = ["sqlx-postgres", "runtime-tokio-rustls",] } serde = { version = "= 1.0.171", features = ["derive", "rc"] } serde_json = "1.0" serde_qs = { version = "0.12", features = ["warp"] } diff --git a/quickwit/quickwit-metastore/Cargo.toml b/quickwit/quickwit-metastore/Cargo.toml index 0dd51313690..26cfa4d9f5c 100644 --- a/quickwit/quickwit-metastore/Cargo.toml +++ b/quickwit/quickwit-metastore/Cargo.toml @@ -24,6 +24,8 @@ serde = { workspace = true } serde_json = { workspace = true } serde_with = { workspace = true } sqlx = { workspace = true, optional = true } +sea-query = { workspace = true } +sea-query-binder = { workspace = true } tempfile = { workspace = true, optional = true } thiserror = { workspace = true } time = { workspace = true } diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index c1fba530a80..bb80e71704d 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -25,7 +25,6 @@ use std::sync::Arc; use std::time::Duration; use async_trait::async_trait; -use itertools::Itertools; use quickwit_common::uri::Uri; use quickwit_common::PrettySample; use quickwit_config::{ @@ -45,6 +44,10 @@ use quickwit_proto::metastore::{ ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::IndexUid; +use sea_query::{ + all, any, Asterisk, Cond, Expr, Func, PostgresQueryBuilder, Query, SelectStatement, +}; +use sea_query_binder::SqlxBinder; use sqlx::migrate::Migrator; use sqlx::postgres::{PgConnectOptions, PgDatabaseError, PgPoolOptions}; use sqlx::{ConnectOptions, Pool, Postgres, Transaction}; @@ -53,7 +56,7 @@ use tracing::log::LevelFilter; use tracing::{debug, error, info, instrument, warn}; use crate::checkpoint::IndexCheckpointDelta; -use crate::metastore::postgresql_model::{PgDeleteTask, PgIndex, PgSplit}; +use crate::metastore::postgresql_model::{PgDeleteTask, PgIndex, PgSplit, Splits, ToTimestampFunc}; use crate::metastore::{instrument_metastore, FilterRange, PublishSplitsRequestExt}; use crate::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, @@ -222,131 +225,124 @@ async fn index_metadata( /// Extends an existing SQL string with the generated filter range appended to the query. /// /// This method is **not** SQL injection proof and should not be used with user-defined values. -fn write_sql_filter( - sql: &mut String, - field_name: impl Display, +fn append_range_filters( + sql: &mut SelectStatement, + field_name: Splits, filter_range: &FilterRange, - value_formatter: impl Fn(&V) -> String, + value_formatter: impl Fn(&V) -> Expr, ) { - match &filter_range.start { - Bound::Included(value) => { - let _ = write!(sql, " AND {} >= {}", field_name, (value_formatter)(value)); - } - Bound::Excluded(value) => { - let _ = write!(sql, " AND {} > {}", field_name, (value_formatter)(value)); - } - Bound::Unbounded => {} + if let Bound::Included(value) = &filter_range.start { + sql.cond_where(Expr::col(field_name).gte((value_formatter)(value))); }; - match &filter_range.end { - Bound::Included(value) => { - let _ = write!(sql, " AND {} <= {}", field_name, (value_formatter)(value)); - } - Bound::Excluded(value) => { - let _ = write!(sql, " AND {} < {}", field_name, (value_formatter)(value)); - } - Bound::Unbounded => {} + if let Bound::Excluded(value) = &filter_range.start { + sql.cond_where(Expr::col(field_name).gt((value_formatter)(value))); + }; + + if let Bound::Included(value) = &filter_range.end { + sql.cond_where(Expr::col(field_name).lte((value_formatter)(value))); + }; + + if let Bound::Excluded(value) = &filter_range.end { + sql.cond_where(Expr::col(field_name).lt((value_formatter)(value))); }; } -fn build_query_filter(mut sql: String, query: &ListSplitsQuery) -> String { +fn append_query_filters(sql: &mut SelectStatement, query: &ListSplitsQuery) { // Note: `ListSplitsQuery` builder enforces a non empty `index_uids` list. - let where_predicate: String = query + + let or_condition = query .index_uids .iter() - .map(|index_uid| format!("index_uid = '{index_uid}'")) - .join(" OR "); - sql.push_str(&format!(" WHERE ({where_predicate})")); + .fold(Cond::any(), |cond, index_uid| { + cond.add(Expr::col(Splits::IndexUid).eq(Expr::val(index_uid.to_string()))) + }); + sql.cond_where(or_condition); if !query.split_states.is_empty() { - let params = query - .split_states - .iter() - .map(|v| format!("'{}'", v.as_str())) - .join(", "); - let _ = write!(sql, " AND split_state IN ({params})"); - } + sql.cond_where( + Expr::col(Splits::SplitState) + .is_in(query.split_states.iter().map(|val| val.to_string())), + ); + }; if let Some(tags) = query.tags.as_ref() { - sql.push_str(" AND ("); - sql.push_str(&tags_filter_expression_helper(tags)); - sql.push(')'); - } + sql.cond_where(tags_filter_expression_helper(tags)); + }; match query.time_range.start { Bound::Included(v) => { - let _ = write!( - sql, - " AND (time_range_end >= {v} OR time_range_end IS NULL)" - ); + sql.cond_where(any![ + Expr::col(Splits::TimeRangeEnd).gte(v), + Expr::col(Splits::TimeRangeEnd).is_null() + ]); } Bound::Excluded(v) => { - let _ = write!(sql, " AND (time_range_end > {v} OR time_range_end IS NULL)"); + sql.cond_where(any![ + Expr::col(Splits::TimeRangeEnd).gt(v), + Expr::col(Splits::TimeRangeEnd).is_null() + ]); } Bound::Unbounded => {} }; match query.time_range.end { Bound::Included(v) => { - let _ = write!( - sql, - " AND (time_range_start <= {v} OR time_range_start IS NULL)" - ); + sql.cond_where(any![ + Expr::col(Splits::TimeRangeStart).lte(v), + Expr::col(Splits::TimeRangeStart).is_null() + ]); } Bound::Excluded(v) => { - let _ = write!( - sql, - " AND (time_range_start < {v} OR time_range_start IS NULL)" - ); + sql.cond_where(any![ + Expr::col(Splits::TimeRangeStart).lt(v), + Expr::col(Splits::TimeRangeStart).is_null() + ]); } Bound::Unbounded => {} }; match &query.mature { Bound::Included(evaluation_datetime) => { - let _ = write!( - sql, - " AND (maturity_timestamp = to_timestamp(0) OR to_timestamp({}) >= \ - maturity_timestamp)", - evaluation_datetime.unix_timestamp() - ); + sql.cond_where(any![ + Expr::col(Splits::MaturityTimestamp) + .eq(Func::cust(ToTimestampFunc).arg(Expr::val(0))), + Expr::col(Splits::MaturityTimestamp).lte( + Func::cust(ToTimestampFunc) + .arg(Expr::val(evaluation_datetime.unix_timestamp())) + ) + ]); } Bound::Excluded(evaluation_datetime) => { - let _ = write!( - sql, - " AND to_timestamp({}) < maturity_timestamp", - evaluation_datetime.unix_timestamp() - ); + sql.cond_where(Expr::col(Splits::MaturityTimestamp).gt( + Func::cust(ToTimestampFunc).arg(Expr::val(evaluation_datetime.unix_timestamp())), + )); } Bound::Unbounded => {} - } - - // WARNING: Not SQL injection proof - write_sql_filter( - &mut sql, - "update_timestamp", + }; + append_range_filters( + sql, + Splits::UpdateTimestamp, &query.update_timestamp, - |val| format!("to_timestamp({val})"), + |&val| Expr::expr(Func::cust(ToTimestampFunc).arg(Expr::val(val))), ); - write_sql_filter( - &mut sql, - "create_timestamp", + append_range_filters( + sql, + Splits::CreateTimestamp, &query.create_timestamp, - |val| format!("to_timestamp({val})"), + |&val| Expr::expr(Func::cust(ToTimestampFunc).arg(Expr::val(val))), ); - write_sql_filter(&mut sql, "delete_opstamp", &query.delete_opstamp, |val| { - val.to_string() + append_range_filters(sql, Splits::DeleteOpstamp, &query.delete_opstamp, |&val| { + Expr::expr(val) }); if let Some(limit) = query.limit { - let _ = write!(sql, " LIMIT {limit}"); + sql.limit(limit as u64); } if let Some(offset) = query.offset { - let _ = write!(sql, " OFFSET {offset}"); + sql.offset(offset as u64); } - - sql } /// Returns the unix timestamp at which the split becomes mature. @@ -813,10 +809,13 @@ impl MetastoreService for PostgresqlMetastore { 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); + let mut sql = Query::select(); + sql.column(Asterisk).from(Splits::Table); + append_query_filters(&mut sql, &query); + + let (sql, values) = sql.build_sqlx(PostgresQueryBuilder); - let pg_splits = sqlx::query_as::<_, PgSplit>(&sql) + let pg_splits = sqlx::query_as_with::<_, PgSplit, _>(&sql, values) .fetch_all(&self.connection_pool) .await?; @@ -1350,35 +1349,38 @@ fn generate_dollar_guard(s: &str) -> String { /// Takes a tag filters AST and returns a sql expression that can be used as /// a filter. -fn tags_filter_expression_helper(tags: &TagFilterAst) -> String { +fn tags_filter_expression_helper(tags: &TagFilterAst) -> Cond { match tags { TagFilterAst::And(child_asts) => { if child_asts.is_empty() { - return "TRUE".to_string(); + return all![Expr::cust("TRUE")]; } - let expr_without_parenthesis = child_asts + + child_asts .iter() .map(tags_filter_expression_helper) - .join(" AND "); - format!("({expr_without_parenthesis})") + .fold(Cond::all(), |cond, child_cond| cond.add(child_cond)) } TagFilterAst::Or(child_asts) => { if child_asts.is_empty() { - return "TRUE".to_string(); + return all![Expr::cust("TRUE")]; } - let expr_without_parenthesis = child_asts + + child_asts .iter() .map(tags_filter_expression_helper) - .join(" OR "); - format!("({expr_without_parenthesis})") + .fold(Cond::any(), |cond, child_cond| cond.add(child_cond)) } + TagFilterAst::Tag { is_present, tag } => { let dollar_guard = generate_dollar_guard(tag); - if *is_present { - format!("${dollar_guard}${tag}${dollar_guard}$ = ANY(tags)") + let expr_str = format!("${dollar_guard}${tag}${dollar_guard}$ = ANY(tags)"); + let expr = if *is_present { + Expr::cust(&expr_str) } else { - format!("NOT (${dollar_guard}${tag}${dollar_guard}$ = ANY(tags))") - } + Expr::cust(&expr_str).not() + }; + all![expr] } } } @@ -1512,10 +1514,12 @@ mod tests { use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; use quickwit_proto::metastore::MetastoreService; use quickwit_proto::types::IndexUid; + use sea_query::{all, any, Asterisk, Cond, Expr, PostgresQueryBuilder, Query}; use time::OffsetDateTime; - use super::{build_query_filter, tags_filter_expression_helper, PostgresqlMetastore}; + use super::{append_query_filters, tags_filter_expression_helper, PostgresqlMetastore}; use crate::metastore::postgresql_metastore::build_index_id_patterns_sql_query; + use crate::metastore::postgresql_model::Splits; use crate::tests::DefaultForTest; use crate::{metastore_test_suite, ListSplitsQuery, SplitState}; @@ -1528,31 +1532,37 @@ mod tests { assert!(metastore.endpoints()[0].protocol().is_postgresql()); } - fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: &str) { + fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: Cond) { assert_eq!(tags_filter_expression_helper(&tags_ast), expected); } #[test] fn test_tags_filter_expression_single_tag() { let tags_ast = tag("my_field:titi"); - test_tags_filter_expression_helper(tags_ast, r#"$$my_field:titi$$ = ANY(tags)"#); + + let expected = all![Expr::cust("$$my_field:titi$$ = ANY(tags)")]; + + test_tags_filter_expression_helper(tags_ast, expected); } #[test] fn test_tags_filter_expression_not_tag() { - test_tags_filter_expression_helper( - no_tag("my_field:titi"), - r#"NOT ($$my_field:titi$$ = ANY(tags))"#, - ); + let expected = all![Expr::cust("$$my_field:titi$$ = ANY(tags)").not()]; + + test_tags_filter_expression_helper(no_tag("my_field:titi"), expected); } #[test] fn test_tags_filter_expression_ands() { let tags_ast = TagFilterAst::And(vec![tag("tag:val1"), tag("tag:val2"), tag("tag:val3")]); - test_tags_filter_expression_helper( - tags_ast, - "($$tag:val1$$ = ANY(tags) AND $$tag:val2$$ = ANY(tags) AND $$tag:val3$$ = ANY(tags))", - ); + + let expected = all![ + Expr::cust("$$tag:val1$$ = ANY(tags)"), + Expr::cust("$$tag:val2$$ = ANY(tags)"), + Expr::cust("$$tag:val3$$ = ANY(tags)"), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); } #[test] @@ -1561,10 +1571,16 @@ mod tests { TagFilterAst::And(vec![tag("tag:val1"), tag("tag:val2")]), tag("tag:val3"), ]); - test_tags_filter_expression_helper( - tags_ast, - "(($$tag:val1$$ = ANY(tags) AND $$tag:val2$$ = ANY(tags)) OR $$tag:val3$$ = ANY(tags))", - ); + + let expected = any![ + all![ + Expr::cust("$$tag:val1$$ = ANY(tags)"), + Expr::cust("$$tag:val2$$ = ANY(tags)"), + ], + Expr::cust("$$tag:val3$$ = ANY(tags)"), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); } #[test] @@ -1573,192 +1589,263 @@ mod tests { TagFilterAst::Or(vec![tag("tag:val1"), tag("tag:val2")]), tag("tag:val3"), ]); - test_tags_filter_expression_helper( - tags_ast, - r#"(($$tag:val1$$ = ANY(tags) OR $$tag:val2$$ = ANY(tags)) AND $$tag:val3$$ = ANY(tags))"#, - ); + + let expected = all![ + any![ + Expr::cust("$$tag:val1$$ = ANY(tags)"), + Expr::cust("$$tag:val2$$ = ANY(tags)"), + ], + Expr::cust("$$tag:val3$$ = ANY(tags)"), + ]; + + test_tags_filter_expression_helper(tags_ast, expected); } #[test] fn test_tags_sql_injection_attempt() { let tags_ast = tag("tag:$$;DELETE FROM something_evil"); - test_tags_filter_expression_helper( - tags_ast, - "$QuickwitGuard$tag:$$;DELETE FROM something_evil$QuickwitGuard$ = ANY(tags)", - ); + + let expected = all![Expr::cust( + "$QuickwitGuard$tag:$$;DELETE FROM something_evil$QuickwitGuard$ = ANY(tags)" + ),]; + + test_tags_filter_expression_helper(tags_ast, expected); + let tags_ast = tag("tag:$QuickwitGuard$;DELETE FROM something_evil"); - test_tags_filter_expression_helper( - tags_ast, + + let expected = all![Expr::cust( "$QuickwitGuardQuickwitGuard$tag:$QuickwitGuard$;DELETE FROM \ - something_evil$QuickwitGuardQuickwitGuard$ = ANY(tags)", - ); + something_evil$QuickwitGuardQuickwitGuard$ = ANY(tags)" + )]; + + test_tags_filter_expression_helper(tags_ast, expected); } #[test] fn test_single_sql_query_builder() { + let mut select_statement = Query::select(); + + let sql = select_statement.column(Asterisk).from(Splits::Table); let index_uid = IndexUid::new_with_random_ulid("test-index"); let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Staged); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); + assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND split_state IN ('Staged')") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "split_state" IN ('Staged')"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_split_state(SplitState::Published); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); + assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND split_state IN ('Published')") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "split_state" IN ('Published')"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()) .with_split_states([SplitState::Published, SplitState::MarkedForDeletion]); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND split_state IN ('Published', \ - 'MarkedForDeletion')" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "split_state" IN ('Published', 'MarkedForDeletion')"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_update_timestamp_lt(51); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND update_timestamp < to_timestamp(51)") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "update_timestamp" < TO_TIMESTAMP(51)"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_create_timestamp_lte(55); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND create_timestamp <= to_timestamp(55)") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "create_timestamp" <= TO_TIMESTAMP(55)"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let maturity_evaluation_datetime = OffsetDateTime::from_unix_timestamp(55).unwrap(); let query = ListSplitsQuery::for_index(index_uid.clone()) .retain_mature(maturity_evaluation_datetime); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); + assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND (maturity_timestamp = to_timestamp(0) OR \ - to_timestamp(55) >= maturity_timestamp)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND ("maturity_timestamp" = TO_TIMESTAMP(0) OR "maturity_timestamp" <= TO_TIMESTAMP(55))"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()) .retain_immature(maturity_evaluation_datetime); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND to_timestamp(55) < maturity_timestamp") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "maturity_timestamp" > TO_TIMESTAMP(55)"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_delete_opstamp_gte(4); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND delete_opstamp >= 4") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "delete_opstamp" >= 4"# + ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_time_range_start_gt(45); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND (time_range_end > 45 OR time_range_end IS \ - NULL)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND ("time_range_end" > 45 OR "time_range_end" IS NULL)"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_time_range_end_lt(45); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND (time_range_start < 45 OR \ - time_range_start IS NULL)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND ("time_range_start" < 45 OR "time_range_start" IS NULL)"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()).with_tags_filter(TagFilterAst::Tag { is_present: false, tag: "tag-2".to_string(), }); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); + assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}') AND (NOT ($$tag-2$$ = ANY(tags)))") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND NOT ($$tag-2$$ = ANY(tags))"# + ) ); } #[test] fn test_combination_sql_query_builder() { + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let index_uid = IndexUid::new_with_random_ulid("test-index"); let query = ListSplitsQuery::for_index(index_uid.clone()) .with_time_range_start_gt(0) .with_time_range_end_lt(40); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND (time_range_end > 0 OR time_range_end IS \ - NULL) AND (time_range_start < 40 OR time_range_start IS NULL)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND ("time_range_end" > 0 OR "time_range_end" IS NULL) AND ("time_range_start" < 40 OR "time_range_start" IS NULL)"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()) .with_time_range_start_gt(45) .with_delete_opstamp_gt(0); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND (time_range_end > 45 OR time_range_end IS \ - NULL) AND delete_opstamp > 0" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND ("time_range_end" > 45 OR "time_range_end" IS NULL) AND "delete_opstamp" > 0"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()) .with_update_timestamp_lt(51) .with_create_timestamp_lte(63); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND update_timestamp < to_timestamp(51) AND \ - create_timestamp <= to_timestamp(63)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND "update_timestamp" < TO_TIMESTAMP(51) AND "create_timestamp" <= TO_TIMESTAMP(63)"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let query = ListSplitsQuery::for_index(index_uid.clone()) .with_time_range_start_gt(90) .with_tags_filter(TagFilterAst::Tag { is_present: true, tag: "tag-1".to_string(), }); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, + sql.to_string(PostgresQueryBuilder), format!( - " WHERE (index_uid = '{index_uid}') AND ($$tag-1$$ = ANY(tags)) AND \ - (time_range_end > 90 OR time_range_end IS NULL)" + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' AND $$tag-1$$ = ANY(tags) AND ("time_range_end" > 90 OR "time_range_end" IS NULL)"# ) ); + let mut select_statement = Query::select(); + let sql = select_statement.column(Asterisk).from(Splits::Table); + let index_uid_2 = IndexUid::new_with_random_ulid("test-index-2"); let query = ListSplitsQuery::try_from_index_uids(vec![index_uid.clone(), index_uid_2.clone()]) .unwrap(); - let sql = build_query_filter(String::new(), &query); + append_query_filters(sql, &query); assert_eq!( - sql, - format!(" WHERE (index_uid = '{index_uid}' OR index_uid = '{index_uid_2}')") + sql.to_string(PostgresQueryBuilder), + format!( + r#"SELECT * FROM "splits" WHERE "index_uid" = '{index_uid}' OR "index_uid" = '{index_uid_2}'"# + ) ); } diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs index 25454dc9261..4dbae9a54a6 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_model.rs @@ -22,6 +22,7 @@ use std::str::FromStr; use quickwit_proto::metastore::{DeleteQuery, DeleteTask, MetastoreError, MetastoreResult}; use quickwit_proto::types::IndexUid; +use sea_query::{Iden, Write}; use tracing::error; use crate::{IndexMetadata, Split, SplitMetadata, SplitState}; @@ -62,6 +63,31 @@ impl PgIndex { } } +#[derive(Iden, Clone, Copy)] +#[allow(dead_code)] +pub enum Splits { + Table, + SplitState, + TimeRangeStart, + TimeRangeEnd, + CreateTimestamp, + UpdateTimestamp, + PublishTimestamp, + MaturityTimestamp, + Tags, + SplitMetadataJson, + IndexUid, + DeleteOpstamp, +} + +pub struct ToTimestampFunc; + +impl Iden for ToTimestampFunc { + fn unquoted(&self, s: &mut dyn Write) { + write!(s, "TO_TIMESTAMP").unwrap() + } +} + /// A model structure for handling split metadata in a database. #[derive(sqlx::FromRow)] pub struct PgSplit { From 02b480af82ab5a2b7098c8df4407607da5638452 Mon Sep 17 00:00:00 2001 From: Paul Masurel Date: Sat, 28 Oct 2023 23:34:55 +0900 Subject: [PATCH 12/12] Making the contracts in Uri a tiny bit clearer. (#3844) * Making the contracts in Uri a tiny bit clearer. Before, the protocol normalization behavior was different depending on whether the Uri was built with `from_well_formed` and `from_str`. Also, `from_well_formed` was used over string passed via gRPC. --------- Co-authored-by: Adrien Guillo --- quickwit/quickwit-cli/src/service.rs | 11 +- quickwit/quickwit-cli/tests/helpers.rs | 21 ++- .../example/src/codegen/hello.rs | 10 +- quickwit/quickwit-codegen/example/src/lib.rs | 8 +- quickwit/quickwit-codegen/src/codegen.rs | 11 +- quickwit/quickwit-common/src/uri.rs | 131 ++++++++---------- .../quickwit-config/src/index_config/mod.rs | 10 +- .../src/index_config/serialize.rs | 2 +- quickwit/quickwit-indexing/src/test_utils.rs | 5 +- .../src/codegen/ingest_service.rs | 2 + .../src/actors/delete_task_planner.rs | 5 +- .../file_backed_metastore_factory.rs | 5 +- .../metastore/file_backed_metastore/mod.rs | 3 +- .../src/metastore/postgresql_metastore.rs | 3 +- .../src/metastore_resolver.rs | 6 +- .../quickwit/quickwit.control_plane.rs | 2 + .../src/codegen/quickwit/quickwit.indexing.rs | 2 + .../quickwit/quickwit.ingest.ingester.rs | 2 + .../quickwit/quickwit.ingest.router.rs | 2 + .../codegen/quickwit/quickwit.metastore.rs | 12 +- quickwit/quickwit-search/src/service.rs | 26 ++-- .../src/index_api/rest_handler.rs | 18 +-- .../src/local_file_storage.rs | 6 +- .../src/object_storage/azure_blob_storage.rs | 12 +- .../object_storage/s3_compatible_storage.rs | 15 +- quickwit/quickwit-storage/src/ram_storage.rs | 14 +- .../quickwit-storage/src/storage_resolver.rs | 9 +- quickwit/quickwit-storage/tests/s3_storage.rs | 5 +- 28 files changed, 175 insertions(+), 183 deletions(-) diff --git a/quickwit/quickwit-cli/src/service.rs b/quickwit/quickwit-cli/src/service.rs index ec0b7b8ee7f..374533633ea 100644 --- a/quickwit/quickwit-cli/src/service.rs +++ b/quickwit/quickwit-cli/src/service.rs @@ -23,7 +23,7 @@ use std::str::FromStr; use clap::{arg, ArgAction, ArgMatches, Command}; use itertools::Itertools; use quickwit_common::runtimes::RuntimesConfig; -use quickwit_common::uri::Uri; +use quickwit_common::uri::{Protocol, Uri}; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; use quickwit_serve::serve_quickwit; @@ -126,11 +126,12 @@ fn quickwit_telemetry_info(config: &NodeConfig) -> QuickwitTelemetryInfo { } // The metastore URI is only relevant if the metastore is enabled. if config.is_service_enabled(QuickwitService::Metastore) { - if config.metastore_uri.protocol().is_postgresql() { - features.insert(QuickwitFeature::PostgresqMetastore); + let feature = if config.metastore_uri.protocol() == Protocol::PostgreSQL { + QuickwitFeature::PostgresqMetastore } else { - features.insert(QuickwitFeature::FileBackedMetastore); - } + QuickwitFeature::FileBackedMetastore + }; + features.insert(feature); } let services = config .enabled_services diff --git a/quickwit/quickwit-cli/tests/helpers.rs b/quickwit/quickwit-cli/tests/helpers.rs index 5c08dafbdc7..ed5eeb6415b 100644 --- a/quickwit/quickwit-cli/tests/helpers.rs +++ b/quickwit/quickwit-cli/tests/helpers.rs @@ -17,9 +17,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::borrow::Borrow; use std::collections::HashMap; use std::fs; use std::path::PathBuf; +use std::str::FromStr; use std::sync::Arc; use anyhow::Context; @@ -186,11 +188,9 @@ pub async fn create_test_env( // TODO: refactor when we have a singleton storage resolver. let metastore_uri = match storage_type { TestStorageType::LocalFileSystem => { - Uri::from_well_formed(format!("file://{}", indexes_dir_path.display())) - } - TestStorageType::S3 => { - Uri::from_well_formed("s3://quickwit-integration-tests/indexes".to_string()) + Uri::from_str(&format!("file://{}", indexes_dir_path.display())).unwrap() } + TestStorageType::S3 => Uri::for_test("s3://quickwit-integration-tests/indexes"), }; let storage_resolver = StorageResolver::unconfigured(); let storage = storage_resolver.resolve(&metastore_uri).await?; @@ -236,11 +236,12 @@ pub async fn create_test_env( resource_files.insert("wiki", wikipedia_docs_path); let config_uri = - Uri::from_well_formed(format!("file://{}", resource_files["config"].display())); - let index_config_uri = Uri::from_well_formed(format!( + Uri::from_str(&format!("file://{}", resource_files["config"].display())).unwrap(); + let index_config_uri = Uri::from_str(&format!( "file://{}", resource_files["index_config"].display() - )); + )) + .unwrap(); let cluster_endpoint = Url::parse(&format!("http://localhost:{rest_listen_port}")) .context("failed to parse cluster endpoint")?; @@ -273,10 +274,8 @@ pub async fn upload_test_file( ) -> PathBuf { let test_data = tokio::fs::read(local_src_path).await.unwrap(); let mut src_location: PathBuf = [r"s3://", bucket, prefix].iter().collect(); - let storage = storage_resolver - .resolve(&Uri::from_well_formed(src_location.to_string_lossy())) - .await - .unwrap(); + let storage_uri = Uri::from_str(src_location.to_string_lossy().borrow()).unwrap(); + let storage = storage_resolver.resolve(&storage_uri).await.unwrap(); storage .put(&PathBuf::from(filename), Box::new(test_data)) .await diff --git a/quickwit/quickwit-codegen/example/src/codegen/hello.rs b/quickwit/quickwit-codegen/example/src/codegen/hello.rs index 8916d34687d..f217ea5606f 100644 --- a/quickwit/quickwit-codegen/example/src/codegen/hello.rs +++ b/quickwit/quickwit-codegen/example/src/codegen/hello.rs @@ -41,6 +41,8 @@ pub struct PingResponse { pub message: ::prost::alloc::string::String, } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; impl PrometheusLabels<1> for HelloRequest { @@ -597,8 +599,8 @@ where } fn endpoints(&self) -> Vec { vec![ - quickwit_common::uri::Uri::from_well_formed(format!("actor://localhost/{}", - self.inner.actor_instance_id())) + quickwit_common::uri::Uri::from_str(& format!("actor://localhost/{}", self + .inner.actor_instance_id())).expect("URI should be valid") ] } } @@ -677,8 +679,8 @@ where self.connection_addrs_rx .borrow() .iter() - .map(|addr| quickwit_common::uri::Uri::from_well_formed( - format!(r"grpc://{}/{}.{}", addr, "hello", "Hello"), + .flat_map(|addr| quickwit_common::uri::Uri::from_str( + &format!("grpc://{addr}/{}.{}", "hello", "Hello"), )) .collect() } diff --git a/quickwit/quickwit-codegen/example/src/lib.rs b/quickwit/quickwit-codegen/example/src/lib.rs index bbe9dd2cb68..e2c3cfd20f7 100644 --- a/quickwit/quickwit-codegen/example/src/lib.rs +++ b/quickwit/quickwit-codegen/example/src/lib.rs @@ -149,6 +149,7 @@ impl Hello for HelloImpl { #[cfg(test)] mod tests { use std::net::SocketAddr; + use std::str::FromStr; use std::sync::atomic::Ordering; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Universe}; @@ -313,7 +314,7 @@ mod tests { grpc_client.check_connectivity().await.unwrap(); assert_eq!( grpc_client.endpoints(), - vec![Uri::from_well_formed("grpc://127.0.0.1:6666/hello.Hello")] + vec![Uri::from_str("grpc://127.0.0.1:6666/hello.Hello").unwrap()] ); // The connectivity check fails if there is no client behind the channel. @@ -403,10 +404,11 @@ mod tests { actor_client.check_connectivity().await.unwrap(); assert_eq!( actor_client.endpoints(), - vec![Uri::from_well_formed(format!( + vec![Uri::from_str(&format!( "actor://localhost/{}", actor_mailbox.actor_instance_id() - ))] + )) + .unwrap()] ); let (ping_stream_tx, ping_stream) = ServiceStream::new_bounded(1); diff --git a/quickwit/quickwit-codegen/src/codegen.rs b/quickwit/quickwit-codegen/src/codegen.rs index ba13eea3d7a..d2f21a43e43 100644 --- a/quickwit/quickwit-codegen/src/codegen.rs +++ b/quickwit/quickwit-codegen/src/codegen.rs @@ -120,10 +120,7 @@ impl CodegenBuilder { pub fn run(self) -> anyhow::Result<()> { ensure!(!self.protos.is_empty(), "proto file list is empty"); ensure!(!self.output_dir.is_empty(), "output directory is undefined"); - ensure!( - !self.result_type_path.is_empty(), - "result type is undefined" - ); + ensure!(!self.result_type_path.is_empty(),); ensure!(!self.error_type_path.is_empty(), "error type is undefined"); Codegen::run(self) @@ -305,6 +302,8 @@ fn generate_all( quote! { // The line below is necessary to opt out of the license header check. /// BEGIN quickwit-codegen + #[allow(unused_imports)] + use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #prom_labels_impl @@ -970,7 +969,7 @@ fn generate_tower_mailbox(context: &CodegenContext) -> TokenStream { } fn endpoints(&self) -> Vec { - vec![quickwit_common::uri::Uri::from_well_formed(format!("actor://localhost/{}", self.inner.actor_instance_id()))] + vec![quickwit_common::uri::Uri::from_str(&format!("actor://localhost/{}", self.inner.actor_instance_id())).expect("URI should be valid")] } } } else { @@ -1114,7 +1113,7 @@ fn generate_grpc_client_adapter(context: &CodegenContext) -> TokenStream { self.connection_addrs_rx .borrow() .iter() - .map(|addr| quickwit_common::uri::Uri::from_well_formed(format!(r"grpc://{}/{}.{}", addr, #grpc_client_package_name_string, #service_name_string))) + .flat_map(|addr| quickwit_common::uri::Uri::from_str(&format!("grpc://{addr}/{}.{}", #grpc_client_package_name_string, #service_name_string))) .collect() } } diff --git a/quickwit/quickwit-common/src/uri.rs b/quickwit/quickwit-common/src/uri.rs index 8c0c74d2404..d0343844d4d 100644 --- a/quickwit/quickwit-common/src/uri.rs +++ b/quickwit/quickwit-common/src/uri.rs @@ -33,14 +33,15 @@ use serde::{Deserialize, Serialize, Serializer}; #[derive(Debug, Clone, Copy, Eq, PartialEq, Hash, Serialize, Deserialize)] #[serde(rename_all = "snake_case")] +#[repr(u8)] pub enum Protocol { - Azure, - File, - Grpc, - Actor, - PostgreSQL, - Ram, - S3, + Actor = 1, + Azure = 2, + File = 3, + Grpc = 4, + PostgreSQL = 5, + Ram = 6, + S3 = 7, } impl Protocol { @@ -56,30 +57,10 @@ impl Protocol { } } - pub fn is_azure(&self) -> bool { - matches!(&self, Protocol::Azure) - } - pub fn is_file(&self) -> bool { matches!(&self, Protocol::File) } - pub fn is_grpc(&self) -> bool { - matches!(&self, Protocol::Grpc) - } - - pub fn is_postgresql(&self) -> bool { - matches!(&self, Protocol::PostgreSQL) - } - - pub fn is_ram(&self) -> bool { - matches!(&self, Protocol::Ram) - } - - pub fn is_s3(&self) -> bool { - matches!(&self, Protocol::S3) - } - pub fn is_file_storage(&self) -> bool { matches!(&self, Protocol::File | Protocol::Ram) } @@ -119,31 +100,30 @@ impl FromStr for Protocol { const PROTOCOL_SEPARATOR: &str = "://"; /// Encapsulates the URI type. +/// +/// URI's string representation are guaranteed to start +/// by the protocol `str()` representation. +/// +/// # Disclaimer +/// +/// Uri has to be built using `Uri::from_str`. +/// This function has some normalization behavior. +/// Some protocol have several acceptable string representation (`pg`, `postgres`, `postgresql`). +/// +/// If the representation in the input string is not canonical, it will get normalized. +/// In other words, a parsed URI may not have the exact string representation as the original +/// string. #[derive(Clone, Eq, PartialEq, Hash)] pub struct Uri { uri: String, - protocol_idx: usize, + protocol: Protocol, } impl Uri { - /// Constructs a [`Uri`] from a properly formatted string `://` where `path` is - /// normalized. Use this method exclusively for trusted input. - pub fn from_well_formed(uri: S) -> Self { - let uri = uri.to_string(); - let protocol_idx = uri.find(PROTOCOL_SEPARATOR).expect( - "URI lacks protocol separator. Use `Uri::from_well_formed` exclusively for trusted \ - input.", - ); - let protocol_str = &uri[..protocol_idx]; - protocol_str.parse::().expect( - "URI protocol is invalid. Use `Uri::from_well_formed` exclusively for trusted input.`", - ); - Self { uri, protocol_idx } - } - - #[cfg(any(test, feature = "testsuite"))] - pub fn for_test(uri: &str) -> Self { - Uri::from_well_formed(uri) + /// This is only used for test. We artificially restrict the lifetime to 'static + /// to avoid misuses. + pub fn for_test(uri: &'static str) -> Self { + Uri::from_str(uri).unwrap() } /// Returns the extension of the URI. @@ -158,7 +138,7 @@ impl Uri { /// Returns the protocol of the URI. pub fn protocol(&self) -> Protocol { - Protocol::from_str(&self.uri[..self.protocol_idx]).expect("Failed to parse URI protocol. This should never happen! Please, report on https://github.com/quickwit-oss/quickwit/issues.") + self.protocol } /// Strips sensitive information such as credentials from URI. @@ -168,7 +148,7 @@ impl Uri { DATABASE_URI_PATTERN .get_or_init(|| { Regex::new("(?P^.*://.*)(?P:.*@)(?P.*)") - .expect("The regular expression should compile.") + .expect("the regular expression should compile") }) .replace(&self.uri, "$before:***redacted***@$after") } else { @@ -181,12 +161,10 @@ impl Uri { } /// Returns the file path of the URI. - /// Applies only to `file://` URIs. + /// Applies only to `file://` and `ram://` URIs. pub fn filepath(&self) -> Option<&Path> { if self.protocol().is_file_storage() { - Some(Path::new( - &self.uri[self.protocol_idx + PROTOCOL_SEPARATOR.len()..], - )) + Some(self.path()) } else { None } @@ -198,32 +176,38 @@ impl Uri { if self.protocol().is_database() { return None; } - let protocol = &self.uri[..self.protocol_idx]; - let path = Path::new(&self.uri[self.protocol_idx + PROTOCOL_SEPARATOR.len()..]); - if self.protocol().is_s3() && path.components().count() < 2 { + let path = self.path(); + let protocol = self.protocol(); + + if protocol == Protocol::S3 && path.components().count() < 2 { return None; } - if self.protocol().is_azure() && path.components().count() < 3 { + if protocol == Protocol::Azure && path.components().count() < 3 { return None; } - path.parent().map(|parent| { - Uri::from_well_formed(format!( - "{protocol}{PROTOCOL_SEPARATOR}{}", - parent.display() - )) + let parent_path = path.parent()?; + + Some(Self { + uri: format!("{protocol}{PROTOCOL_SEPARATOR}{}", parent_path.display()), + protocol, }) } + fn path(&self) -> &Path { + Path::new(&self.uri[self.protocol.as_str().len() + PROTOCOL_SEPARATOR.len()..]) + } + /// Returns the last component of the URI. pub fn file_name(&self) -> Option<&Path> { - if self.protocol().is_postgresql() { + if self.protocol() == Protocol::PostgreSQL { return None; } - let path = Path::new(&self.uri[self.protocol_idx + PROTOCOL_SEPARATOR.len()..]); - if self.protocol().is_s3() && path.components().count() < 2 { + let path = self.path(); + + if self.protocol() == Protocol::S3 && path.components().count() < 2 { return None; } - if self.protocol().is_azure() && path.components().count() < 3 { + if self.protocol() == Protocol::Azure && path.components().count() < 3 { return None; } path.file_name().map(Path::new) @@ -263,7 +247,7 @@ impl Uri { }; Ok(Self { uri: joined, - protocol_idx: self.protocol_idx, + protocol: self.protocol, }) } @@ -309,7 +293,7 @@ impl Uri { } Ok(Self { uri: format!("{protocol}{PROTOCOL_SEPARATOR}{path}"), - protocol_idx: protocol.as_str().len(), + protocol, }) } } @@ -730,19 +714,20 @@ mod tests { ); assert_eq!( Uri::for_test("postgres://localhost:5432/metastore").as_redacted_str(), - "postgres://localhost:5432/metastore" + "postgresql://localhost:5432/metastore" ); assert_eq!( - Uri::for_test("postgres://username@localhost:5432/metastore").as_redacted_str(), - "postgres://username@localhost:5432/metastore" + Uri::for_test("pg://username@localhost:5432/metastore").as_redacted_str(), + "postgresql://username@localhost:5432/metastore" ); { for protocol in ["postgres", "postgresql"] { - let uri = Uri::from_well_formed(format!( + let uri = Uri::from_str(&format!( "{protocol}://username:password@localhost:5432/metastore" - )); + )) + .unwrap(); let expected_uri = - format!("{protocol}://username:***redacted***@localhost:5432/metastore"); + "postgresql://username:***redacted***@localhost:5432/metastore".to_string(); assert_eq!(uri.as_redacted_str(), expected_uri); assert_eq!(format!("{uri}"), expected_uri); assert_eq!( diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index 35fbcd5dd15..68654049593 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -480,7 +480,7 @@ impl TestableForRegression for IndexConfig { }; IndexConfig { index_id: "my-index".to_string(), - index_uri: Uri::from_well_formed("s3://quickwit-indexes/my-index"), + index_uri: Uri::for_test("s3://quickwit-indexes/my-index"), doc_mapping, indexing_settings, retention_policy, @@ -559,7 +559,7 @@ mod tests { let index_config = load_index_config_from_user_config( config_format, file.as_bytes(), - &Uri::from_well_formed("s3://defaultbucket/"), + &Uri::for_test("s3://defaultbucket/"), ) .unwrap(); assert_eq!(index_config.doc_mapping.tokenizers.len(), 1); @@ -638,7 +638,7 @@ mod tests { #[test] fn test_index_config_default_values() { - let default_index_root_uri = Uri::from_well_formed("s3://defaultbucket/"); + let default_index_root_uri = Uri::for_test("s3://defaultbucket/"); { let index_config_filepath = get_index_config_filepath("minimal-hdfs-logs.yaml"); let file_content = std::fs::read_to_string(index_config_filepath).unwrap(); @@ -721,7 +721,7 @@ mod tests { let minimal_config: IndexConfig = load_index_config_from_user_config( ConfigFormat::Yaml, config_yaml.as_bytes(), - &Uri::from_well_formed("s3://my-index"), + &Uri::for_test("s3://my-index"), ) .unwrap(); assert_eq!( @@ -745,7 +745,7 @@ mod tests { let parsing_config_error = load_index_config_from_user_config( ConfigFormat::Yaml, config_yaml.as_bytes(), - &Uri::from_well_formed("s3://my-index"), + &Uri::for_test("s3://my-index"), ) .unwrap_err(); println!("{parsing_config_error:?}"); diff --git a/quickwit/quickwit-config/src/index_config/serialize.rs b/quickwit/quickwit-config/src/index_config/serialize.rs index 30733c1edbc..373afa632a9 100644 --- a/quickwit/quickwit-config/src/index_config/serialize.rs +++ b/quickwit/quickwit-config/src/index_config/serialize.rs @@ -249,7 +249,7 @@ mod test { ConfigFormat::Yaml, config_yaml.as_bytes(), // same but without the trailing slash. - &Uri::from_well_formed("s3://mybucket"), + &Uri::for_test("s3://mybucket"), ) .unwrap(); assert_eq!(index_config.index_uri.as_str(), "s3://mybucket/hdfs-logs"); diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 050a062cc29..784cc1b60be 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -18,6 +18,7 @@ // along with this program. If not, see . use std::num::NonZeroUsize; +use std::str::FromStr; use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; @@ -65,7 +66,7 @@ pub struct TestSandbox { const METASTORE_URI: &str = "ram://quickwit-test-indexes"; fn index_uri(index_id: &str) -> Uri { - Uri::from_well_formed(format!("{METASTORE_URI}/{index_id}")) + Uri::from_str(&format!("{METASTORE_URI}/{index_id}")).unwrap() } impl TestSandbox { @@ -99,7 +100,7 @@ impl TestSandbox { let metastore_resolver = MetastoreResolver::configured(storage_resolver.clone(), &MetastoreConfigs::default()); let mut metastore = metastore_resolver - .resolve(&Uri::from_well_formed(METASTORE_URI)) + .resolve(&Uri::for_test(METASTORE_URI)) .await?; let create_index_request = CreateIndexRequest::try_from_index_config(index_config.clone())?; let index_uid: IndexUid = metastore diff --git a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs index 2d85fa58ec2..b2954db920f 100644 --- a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs +++ b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs @@ -151,6 +151,8 @@ impl CommitType { } } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index 0524d07726a..d51e8274859 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -18,8 +18,10 @@ // along with this program. If not, see . use std::collections::{HashMap, HashSet}; +use std::str::FromStr; use std::time::Duration; +use anyhow::Context; use async_trait::async_trait; use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, QueueCapacity}; @@ -313,11 +315,12 @@ impl DeleteTaskPlanner { ..Default::default() }; let mut search_indexes_metas = HashMap::new(); + let index_uri = Uri::from_str(index_uri).context("invalid index URI")?; search_indexes_metas.insert( IndexUid::from(delete_query.index_uid.clone()), IndexMetasForLeafSearch { doc_mapper_str: doc_mapper_str.to_string(), - index_uri: Uri::from_well_formed(index_uri), + index_uri, }, ); let leaf_search_request = jobs_to_leaf_requests( 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 6b174f7bc9a..20ef19bdc1f 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,6 +18,7 @@ // along with this program. If not, see . use std::collections::HashMap; +use std::str::FromStr; use std::sync::Arc; use std::time::Duration; @@ -111,7 +112,9 @@ impl MetastoreFactory for FileBackedMetastoreFactory { uri: &Uri, ) -> Result { let (uri_stripped, polling_interval_opt) = extract_polling_interval_from_uri(uri.as_str()); - let uri = Uri::from_well_formed(uri_stripped); + let uri = Uri::from_str(&uri_stripped).map_err(|_| { + MetastoreResolverError::InvalidConfig(format!("invalid URI: `{uri_stripped}`")) + })?; if let Some(metastore) = self.get_from_cache(&uri).await { debug!("using metastore from cache"); return Ok(metastore); 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 b4aa4b446ab..e86e02dce6d 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -961,6 +961,7 @@ mod tests { use std::sync::Arc; use futures::executor::block_on; + use quickwit_common::uri::Protocol; use quickwit_config::IndexConfig; use quickwit_proto::metastore::{DeleteQuery, MetastoreError}; use quickwit_query::query_ast::qast_helper; @@ -983,7 +984,7 @@ mod tests { 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()); + assert_eq!(metastore.endpoints()[0].protocol(), Protocol::Ram); } #[tokio::test] diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index bb80e71704d..c9fee14efc9 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -1511,6 +1511,7 @@ impl crate::tests::DefaultForTest for PostgresqlMetastore { #[cfg(test)] mod tests { + use quickwit_common::uri::Protocol; use quickwit_doc_mapper::tag_pruning::{no_tag, tag, TagFilterAst}; use quickwit_proto::metastore::MetastoreService; use quickwit_proto::types::IndexUid; @@ -1529,7 +1530,7 @@ mod tests { 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()); + assert_eq!(metastore.endpoints()[0].protocol(), Protocol::PostgreSQL); } fn test_tags_filter_expression_helper(tags_ast: TagFilterAst, expected: Cond) { diff --git a/quickwit/quickwit-metastore/src/metastore_resolver.rs b/quickwit/quickwit-metastore/src/metastore_resolver.rs index b03cc6ba11b..5e0eb7f8aea 100644 --- a/quickwit/quickwit-metastore/src/metastore_resolver.rs +++ b/quickwit/quickwit-metastore/src/metastore_resolver.rs @@ -176,6 +176,8 @@ impl MetastoreResolverBuilder { #[cfg(test)] mod tests { + use std::str::FromStr; + use super::*; #[tokio::test] @@ -183,7 +185,7 @@ mod tests { let metastore_resolver = MetastoreResolver::unconfigured(); let tmp_dir = tempfile::tempdir().unwrap(); let metastore_filepath = format!("file://{}/metastore", tmp_dir.path().display()); - let metastore_uri = Uri::from_well_formed(metastore_filepath); + let metastore_uri = Uri::from_str(&metastore_filepath).unwrap(); metastore_resolver.resolve(&metastore_uri).await.unwrap(); } @@ -199,7 +201,7 @@ mod tests { }); let (_uri_protocol, uri_path) = test_database_url.split_once("://").unwrap(); for protocol in &["postgres", "postgresql"] { - let postgres_uri = Uri::from_well_formed(format!("{protocol}://{uri_path}")); + let postgres_uri = Uri::from_str(&format!("{protocol}://{uri_path}")).unwrap(); metastore_resolver.resolve(&postgres_uri).await.unwrap(); } } 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 410f1a9ba65..82535245522 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -49,6 +49,8 @@ pub struct GetOpenShardsSubresponse { pub open_shards: ::prost::alloc::vec::Vec, } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index 8ea5cb026e9..7b9c977eac2 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -24,6 +24,8 @@ pub struct IndexingTask { #[derive(Clone, PartialEq, ::prost::Message)] pub struct ApplyIndexingPlanResponse {} /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] 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 72cc0d36a57..70258aa0ecf 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -310,6 +310,8 @@ impl PersistFailureReason { } } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; pub type IngesterServiceStream = quickwit_common::ServiceStream< crate::ingest::IngestV2Result, 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 49865e6e2b3..5b26c86fecf 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -92,6 +92,8 @@ impl IngestFailureReason { } } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 9eb600aef13..39cfbebce99 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -439,6 +439,8 @@ impl SourceType { } } /// BEGIN quickwit-codegen +#[allow(unused_imports)] +use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; impl PrometheusLabels<1> for CreateIndexRequest { @@ -2975,8 +2977,8 @@ where } fn endpoints(&self) -> Vec { vec![ - quickwit_common::uri::Uri::from_well_formed(format!("actor://localhost/{}", - self.inner.actor_instance_id())) + quickwit_common::uri::Uri::from_str(& format!("actor://localhost/{}", self + .inner.actor_instance_id())).expect("URI should be valid") ] } } @@ -3244,10 +3246,8 @@ where self.connection_addrs_rx .borrow() .iter() - .map(|addr| quickwit_common::uri::Uri::from_well_formed( - format!( - r"grpc://{}/{}.{}", addr, "quickwit.metastore", "MetastoreService" - ), + .flat_map(|addr| quickwit_common::uri::Uri::from_str( + &format!("grpc://{addr}/{}.{}", "quickwit.metastore", "MetastoreService"), )) .collect() } diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index f75d200bc34..ce28e2962a1 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -182,12 +182,10 @@ impl SearchService for SearchServiceImpl { ) -> crate::Result { let search_request: Arc = leaf_search_request .search_request - .ok_or_else(|| SearchError::Internal("no search request.".to_string()))? + .ok_or_else(|| SearchError::Internal("no search request".to_string()))? .into(); - let storage = self - .storage_resolver - .resolve(&Uri::from_well_formed(leaf_search_request.index_uri)) - .await?; + let index_uri = Uri::from_str(&leaf_search_request.index_uri)?; + let storage = self.storage_resolver.resolve(&index_uri).await?; let doc_mapper = deserialize_doc_mapper(&leaf_search_request.doc_mapper)?; let leaf_search_response = leaf_search( @@ -206,10 +204,8 @@ impl SearchService for SearchServiceImpl { &self, fetch_docs_request: FetchDocsRequest, ) -> crate::Result { - let storage = self - .storage_resolver - .resolve(&Uri::from_well_formed(fetch_docs_request.index_uri)) - .await?; + let index_uri = Uri::from_str(&fetch_docs_request.index_uri)?; + let storage = self.storage_resolver.resolve(&index_uri).await?; let snippet_request_opt: Option<&SnippetRequest> = fetch_docs_request.snippet_request.as_ref(); let doc_mapper = deserialize_doc_mapper(&fetch_docs_request.doc_mapper)?; @@ -246,10 +242,8 @@ impl SearchService for SearchServiceImpl { let stream_request = leaf_stream_request .request .ok_or_else(|| SearchError::Internal("no search request".to_string()))?; - let storage = self - .storage_resolver - .resolve(&Uri::from_well_formed(leaf_stream_request.index_uri)) - .await?; + let index_uri = Uri::from_str(&leaf_stream_request.index_uri)?; + let storage = self.storage_resolver.resolve(&index_uri).await?; let doc_mapper = deserialize_doc_mapper(&leaf_stream_request.doc_mapper)?; let leaf_receiver = leaf_search_stream( self.searcher_context.clone(), @@ -283,10 +277,8 @@ impl SearchService for SearchServiceImpl { let search_request = leaf_search_request .list_terms_request .ok_or_else(|| SearchError::Internal("no search request".to_string()))?; - let storage = self - .storage_resolver - .resolve(&Uri::from_well_formed(leaf_search_request.index_uri)) - .await?; + let index_uri = Uri::from_str(&leaf_search_request.index_uri)?; + let storage = self.storage_resolver.resolve(&index_uri).await?; let split_ids = leaf_search_request.split_offsets; let leaf_search_response = leaf_list_terms( diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index b5c0b4c9545..4deba4d38d4 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -1341,8 +1341,7 @@ mod tests { let 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)); { @@ -1382,8 +1381,7 @@ mod tests { 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)); let resp = warp::test::request() @@ -1505,8 +1503,7 @@ mod tests { let 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)) .recover(recover_fn); @@ -1528,8 +1525,7 @@ mod tests { let 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)) .recover(recover_fn); @@ -1567,8 +1563,7 @@ mod tests { let 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)) .recover(recover_fn); @@ -1604,8 +1599,7 @@ mod tests { let 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 = - Uri::from_well_formed("file:///default-index-root-uri"); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); let index_management_handler = super::index_management_handlers(index_service, Arc::new(node_config)) .recover(recover_fn); diff --git a/quickwit/quickwit-storage/src/local_file_storage.rs b/quickwit/quickwit-storage/src/local_file_storage.rs index 1124aadc317..39e0f11d513 100644 --- a/quickwit/quickwit-storage/src/local_file_storage.rs +++ b/quickwit/quickwit-storage/src/local_file_storage.rs @@ -402,20 +402,20 @@ mod tests { async fn test_local_file_storage_factory() -> anyhow::Result<()> { let temp_dir = tempfile::tempdir()?; let index_uri = - Uri::from_well_formed(format!("file://{}/foo/bar", temp_dir.path().display())); + Uri::from_str(&format!("file://{}/foo/bar", temp_dir.path().display())).unwrap(); let local_file_storage_factory = LocalFileStorageFactory; let local_file_storage = local_file_storage_factory.resolve(&index_uri).await?; assert_eq!(local_file_storage.uri(), &index_uri); let err = local_file_storage_factory - .resolve(&Uri::from_well_formed("s3://foo/bar")) + .resolve(&Uri::for_test("s3://foo/bar")) .await .err() .unwrap(); assert!(matches!(err, StorageResolverError::InvalidUri { .. })); let err = local_file_storage_factory - .resolve(&Uri::from_well_formed("s3://")) + .resolve(&Uri::for_test("s3://")) .await .err() .unwrap(); diff --git a/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs b/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs index 11b444a9e4a..489caddcc75 100644 --- a/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs +++ b/quickwit/quickwit-storage/src/object_storage/azure_blob_storage.rs @@ -131,10 +131,12 @@ impl AzureBlobStorage { /// Creates an emulated storage for testing. #[cfg(feature = "testsuite")] pub fn new_emulated(container: &str) -> Self { + use std::str::FromStr; + let container_client = ClientBuilder::emulator().container_client(container); Self { container_client, - uri: Uri::from_well_formed(format!("azure://tester/{container}")), + uri: Uri::from_str(&format!("azure://tester/{container}")).unwrap(), prefix: PathBuf::new(), multipart_policy: MultiPartPolicy::default(), retry_params: RetryParams { @@ -619,20 +621,20 @@ mod tests { #[test] fn test_parse_azure_uri() { - assert!(parse_azure_uri(&Uri::from_well_formed("azure://")).is_none()); + assert!(parse_azure_uri(&Uri::for_test("azure://")).is_none()); let (container, prefix) = - parse_azure_uri(&Uri::from_well_formed("azure://test-container")).unwrap(); + parse_azure_uri(&Uri::for_test("azure://test-container")).unwrap(); assert_eq!(container, "test-container"); assert!(prefix.to_str().unwrap().is_empty()); let (container, prefix) = - parse_azure_uri(&Uri::from_well_formed("azure://test-container/")).unwrap(); + parse_azure_uri(&Uri::for_test("azure://test-container/")).unwrap(); assert_eq!(container, "test-container"); assert!(prefix.to_str().unwrap().is_empty()); let (container, prefix) = - parse_azure_uri(&Uri::from_well_formed("azure://test-container/indexes")).unwrap(); + parse_azure_uri(&Uri::for_test("azure://test-container/indexes")).unwrap(); assert_eq!(container, "test-container"); assert_eq!(prefix.to_str().unwrap(), "indexes"); } diff --git a/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs b/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs index eca7af6d3c2..cbca48b5ad6 100644 --- a/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs +++ b/quickwit/quickwit-storage/src/object_storage/s3_compatible_storage.rs @@ -873,29 +873,26 @@ mod tests { #[test] fn test_parse_uri() { assert_eq!( - parse_s3_uri(&Uri::from_well_formed("s3://bucket/path/to/object")), + parse_s3_uri(&Uri::for_test("s3://bucket/path/to/object")), Some(("bucket".to_string(), PathBuf::from("path/to/object"))) ); assert_eq!( - parse_s3_uri(&Uri::from_well_formed("s3://bucket/path")), + parse_s3_uri(&Uri::for_test("s3://bucket/path")), Some(("bucket".to_string(), PathBuf::from("path"))) ); assert_eq!( - parse_s3_uri(&Uri::from_well_formed("s3://bucket/path/to/object")), + parse_s3_uri(&Uri::for_test("s3://bucket/path/to/object")), Some(("bucket".to_string(), PathBuf::from("path/to/object"))) ); assert_eq!( - parse_s3_uri(&Uri::from_well_formed("s3://bucket/")), + parse_s3_uri(&Uri::for_test("s3://bucket/")), Some(("bucket".to_string(), PathBuf::from(""))) ); assert_eq!( - parse_s3_uri(&Uri::from_well_formed("s3://bucket")), + parse_s3_uri(&Uri::for_test("s3://bucket")), Some(("bucket".to_string(), PathBuf::from(""))) ); - assert_eq!( - parse_s3_uri(&Uri::from_well_formed("ram://path/to/file")), - None - ); + assert_eq!(parse_s3_uri(&Uri::for_test("ram://path/to/file")), None); } #[tokio::test] diff --git a/quickwit/quickwit-storage/src/ram_storage.rs b/quickwit/quickwit-storage/src/ram_storage.rs index 047992551af..31243e9ee8c 100644 --- a/quickwit/quickwit-storage/src/ram_storage.rs +++ b/quickwit/quickwit-storage/src/ram_storage.rs @@ -25,7 +25,7 @@ use std::path::{Path, PathBuf}; use std::sync::Arc; use async_trait::async_trait; -use quickwit_common::uri::Uri; +use quickwit_common::uri::{Protocol, Uri}; use quickwit_config::StorageBackend; use tokio::io::{AsyncRead, AsyncWriteExt}; use tokio::sync::RwLock; @@ -58,7 +58,7 @@ impl fmt::Debug for RamStorage { impl Default for RamStorage { fn default() -> Self { Self { - uri: Uri::from_well_formed("ram:///"), + uri: Uri::for_test("ram:///"), files: Arc::new(RwLock::new(HashMap::new())), } } @@ -179,7 +179,7 @@ impl RamStorageBuilder { /// Finalizes the [`RamStorage`] creation. pub fn build(self) -> RamStorage { RamStorage { - uri: Uri::from_well_formed("ram:///"), + uri: Uri::for_test("ram:///"), files: Arc::new(RwLock::new(self.files)), } } @@ -206,7 +206,7 @@ impl StorageFactory for RamStorageFactory { async fn resolve(&self, uri: &Uri) -> Result, StorageResolverError> { match uri.filepath() { - Some(prefix) if uri.protocol().is_ram() => Ok(add_prefix_to_storage( + Some(prefix) if uri.protocol() == Protocol::Ram => Ok(add_prefix_to_storage( self.ram_storage.clone(), prefix.to_path_buf(), uri.clone(), @@ -235,13 +235,13 @@ mod tests { #[tokio::test] async fn test_ram_storage_factory() { let ram_storage_factory = RamStorageFactory::default(); - let ram_uri = Uri::from_well_formed("s3:///foo"); + let ram_uri = Uri::for_test("s3:///foo"); let err = ram_storage_factory.resolve(&ram_uri).await.err().unwrap(); assert!(matches!(err, StorageResolverError::InvalidUri { .. })); - let data_uri = Uri::from_well_formed("ram:///data"); + let data_uri = Uri::for_test("ram:///data"); let data_storage = ram_storage_factory.resolve(&data_uri).await.ok().unwrap(); - let home_uri = Uri::from_well_formed("ram:///home"); + let home_uri = Uri::for_test("ram:///home"); let home_storage = ram_storage_factory.resolve(&home_uri).await.ok().unwrap(); assert_ne!(data_storage.uri(), home_storage.uri()); diff --git a/quickwit/quickwit-storage/src/storage_resolver.rs b/quickwit/quickwit-storage/src/storage_resolver.rs index 9cd79c3d4b7..7971ff8d3af 100644 --- a/quickwit/quickwit-storage/src/storage_resolver.rs +++ b/quickwit/quickwit-storage/src/storage_resolver.rs @@ -178,9 +178,7 @@ mod tests { .register(ram_storage_factory) .build() .unwrap(); - let storage = storage_resolver - .resolve(&Uri::from_well_formed("ram:///".to_string())) - .await?; + let storage = storage_resolver.resolve(&Uri::for_test("ram:///")).await?; let data = storage.get_all(Path::new("hello")).await?; assert_eq!(&data[..], b"hello_content_second"); Ok(()) @@ -213,7 +211,7 @@ mod tests { .build() .unwrap(); let storage = storage_resolver - .resolve(&Uri::from_well_formed("ram:///home".to_string())) + .resolve(&Uri::for_test("ram:///home")) .await?; let data = storage.get_all(Path::new("hello")).await?; assert_eq!(&data[..], b"hello_content_second"); @@ -223,8 +221,7 @@ mod tests { #[tokio::test] async fn test_storage_resolver_unsupported_protocol() { let storage_resolver = StorageResolver::unconfigured(); - let storage_uri = - Uri::from_well_formed("postgresql://localhost:5432/metastore".to_string()); + let storage_uri = Uri::for_test("postgresql://localhost:5432/metastore"); let resolver_error = storage_resolver.resolve(&storage_uri).await.unwrap_err(); assert!(matches!( resolver_error, diff --git a/quickwit/quickwit-storage/tests/s3_storage.rs b/quickwit/quickwit-storage/tests/s3_storage.rs index 40ba7c792c5..c40a8fbabd2 100644 --- a/quickwit/quickwit-storage/tests/s3_storage.rs +++ b/quickwit/quickwit-storage/tests/s3_storage.rs @@ -21,6 +21,7 @@ // makes it possible to connect to Amazon S3's quickwit-integration-test bucket. use std::path::PathBuf; +use std::str::FromStr; use anyhow::Context; use once_cell::sync::OnceCell; @@ -52,7 +53,7 @@ fn test_runtime_singleton() -> &'static Runtime { async fn run_s3_storage_test_suite(s3_storage_config: S3StorageConfig, bucket_uri: &str) { setup_logging_for_tests(); - let storage_uri = Uri::from_well_formed(bucket_uri); + let storage_uri = Uri::from_str(bucket_uri).unwrap(); let mut object_storage = S3CompatibleObjectStorage::from_uri(&s3_storage_config, &storage_uri) .await .unwrap(); @@ -124,7 +125,7 @@ fn test_suite_on_s3_storage_bulk_delete_single_object_delete_api() { let bucket_uri = append_random_suffix( "s3://quickwit-integration-tests/test-bulk-delete-single-object-delete-api", ); - let storage_uri = Uri::from_well_formed(bucket_uri); + let storage_uri = Uri::from_str(&bucket_uri).unwrap(); let test_runtime = test_runtime_singleton(); test_runtime.block_on(async move { let mut object_storage =