From 6c315e771e1624326ccbd08d2abf364235a7816e Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Mon, 2 Oct 2023 18:11:49 -0400 Subject: [PATCH] WIP --- quickwit/Cargo.lock | 5 +- quickwit/Cargo.toml | 3 +- quickwit/quickwit-codegen/example/src/lib.rs | 1 - quickwit/quickwit-common/Cargo.toml | 1 + quickwit/quickwit-common/src/tower/buffer.rs | 5 +- quickwit/quickwit-common/src/tower/pool.rs | 129 ++-- .../quickwit-config/src/index_config/mod.rs | 2 +- quickwit/quickwit-control-plane/Cargo.toml | 1 + .../src/control_plane.rs | 40 +- .../src/ingest/ingest_controller.rs | 308 +++++----- quickwit/quickwit-control-plane/src/lib.rs | 26 +- .../quickwit-control-plane/src/scheduler.rs | 8 +- .../src/default_doc_mapper/default_mapper.rs | 6 +- .../src/default_doc_mapper/mapping_tree.rs | 11 +- .../src/default_doc_mapper/mod.rs | 2 +- .../quickwit-doc-mapper/src/doc_mapper.rs | 8 +- .../quickwit-index-management/src/index.rs | 4 +- .../src/actors/indexing_pipeline.rs | 12 +- .../quickwit-indexing/src/actors/publisher.rs | 6 +- .../quickwit-indexing/src/actors/uploader.rs | 2 +- .../merge_policy/stable_log_merge_policy.rs | 1 + .../src/source/ingest/mod.rs | 17 +- quickwit/quickwit-ingest/src/errors.rs | 2 +- .../quickwit-ingest/src/ingest_v2/fetch.rs | 5 +- quickwit/quickwit-ingest/src/ingest_v2/gc.rs | 225 +++++++ .../src/ingest_v2/ingest_metastore.rs | 43 ++ .../quickwit-ingest/src/ingest_v2/ingester.rs | 574 ++++++++++++------ quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 9 +- .../quickwit-ingest/src/ingest_v2/models.rs | 278 +++++++++ .../src/ingest_v2/replication.rs | 23 +- .../quickwit-ingest/src/ingest_v2/router.rs | 266 +++++--- .../src/ingest_v2/shard_table.rs | 45 +- .../src/ingest_v2/test_utils.rs | 3 +- .../quickwit-jaeger/src/integration_tests.rs | 4 +- quickwit/quickwit-janitor/src/error.rs | 4 +- .../src/metastore/control_plane_metastore.rs | 8 +- .../file_backed_index/shards.rs | 28 +- .../quickwit-opentelemetry/src/otlp/traces.rs | 4 +- .../protos/quickwit/control_plane.proto | 52 +- .../protos/quickwit/ingester.proto | 7 +- .../protos/quickwit/metastore.proto | 12 +- .../quickwit/quickwit.control_plane.rs | 488 +++++++++++---- .../quickwit/quickwit.ingest.ingester.rs | 29 +- .../codegen/quickwit/quickwit.metastore.rs | 40 +- .../quickwit-proto/src/control_plane/mod.rs | 28 +- quickwit/quickwit-proto/src/error.rs | 6 +- quickwit/quickwit-proto/src/indexing/mod.rs | 2 +- quickwit/quickwit-proto/src/ingest/mod.rs | 2 +- quickwit/quickwit-proto/src/lib.rs | 4 +- .../quickwit-proto/src/metastore/events.rs | 5 +- quickwit/quickwit-proto/src/metastore/mod.rs | 20 +- quickwit/quickwit-proto/src/types.rs | 40 ++ .../src/query_ast/field_presence.rs | 2 +- .../quickwit-search/src/cluster_client.rs | 6 +- quickwit/quickwit-search/src/error.rs | 2 +- quickwit/quickwit-search/src/lib.rs | 4 +- .../quickwit-search/src/search_job_placer.rs | 12 +- .../quickwit-search/src/search_stream/leaf.rs | 2 +- quickwit/quickwit-search/src/tests.rs | 4 +- .../src/elastic_search_api/bulk.rs | 4 +- .../src/elastic_search_api/model/error.rs | 2 +- .../quickwit-serve/src/ingest_metastore.rs | 65 ++ .../quickwit-serve/src/json_api_response.rs | 6 +- quickwit/quickwit-serve/src/lib.rs | 13 +- quickwit/quickwit-serve/src/search_api/mod.rs | 4 +- .../src/search_api/rest_handler.rs | 2 +- quickwit/quickwit-serve/src/ui_handler.rs | 2 +- .../src/local_file_storage.rs | 2 +- quickwit/rust-toolchain.toml | 2 +- 69 files changed, 2124 insertions(+), 864 deletions(-) create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/gc.rs create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs create mode 100644 quickwit/quickwit-ingest/src/ingest_v2/models.rs create mode 100644 quickwit/quickwit-serve/src/ingest_metastore.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index e0b66058d58..673f50a8ee4 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -3696,8 +3696,7 @@ dependencies = [ [[package]] name = "mrecordlog" version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ae279fda279177cadf1f2222ce4462fef46d814ba564e094c0b03e6503e39e0" +source = "git+https://github.com/quickwit-oss/mrecordlog?rev=ae1ca3f#ae1ca3f08fdfd721feec7a6bfab481df5ddb7f75" dependencies = [ "async-trait", "bytes", @@ -5072,6 +5071,7 @@ dependencies = [ "async-speed-limit", "async-trait", "byte-unit", + "dashmap", "dyn-clone", "env_logger", "fnv", @@ -5140,6 +5140,7 @@ dependencies = [ "async-trait", "chitchat", "dyn-clone", + "fnv", "futures", "http", "hyper", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 3e963abc83e..799c5179ecb 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -55,6 +55,7 @@ colored = "2.0.0" console-subscriber = "0.1.8" criterion = { version = "0.5", features = ["async_tokio"] } cron = "0.12.0" +dashmap = "5.5.3" dialoguer = "0.10.3" dotenv = "0.15" dyn-clone = "1.0.10" @@ -100,7 +101,7 @@ matches = "0.1.9" md5 = "0.7" mime_guess = "2.0.4" mockall = "0.11" -mrecordlog = "0.4" +mrecordlog = { git = "https://github.com/quickwit-oss/mrecordlog", rev = "ae1ca3f" } new_string_template = "1.4.0" nom = "7.1.3" num_cpus = "1" diff --git a/quickwit/quickwit-codegen/example/src/lib.rs b/quickwit/quickwit-codegen/example/src/lib.rs index 62aa573f67b..16e3e67a153 100644 --- a/quickwit/quickwit-codegen/example/src/lib.rs +++ b/quickwit/quickwit-codegen/example/src/lib.rs @@ -34,7 +34,6 @@ use tower::{Layer, Service}; pub use crate::error::HelloError; pub use crate::hello::*; -use crate::hello::{Hello, HelloRequest, HelloResponse}; pub type HelloResult = Result; diff --git a/quickwit/quickwit-common/Cargo.toml b/quickwit/quickwit-common/Cargo.toml index a3666468711..80e6c54475a 100644 --- a/quickwit/quickwit-common/Cargo.toml +++ b/quickwit/quickwit-common/Cargo.toml @@ -14,6 +14,7 @@ anyhow = { workspace = true } async-speed-limit = { workspace = true } async-trait = { workspace = true } byte-unit = { workspace = true } +dashmap = { workspace = true } dyn-clone = { workspace = true } env_logger = { workspace = true } fnv = { workspace = true } diff --git a/quickwit/quickwit-common/src/tower/buffer.rs b/quickwit/quickwit-common/src/tower/buffer.rs index 1d4e8b4c571..aa5ba7b39cc 100644 --- a/quickwit/quickwit-common/src/tower/buffer.rs +++ b/quickwit/quickwit-common/src/tower/buffer.rs @@ -166,10 +166,7 @@ impl fmt::Debug for BufferLayer { impl Clone for BufferLayer { fn clone(&self) -> Self { - Self { - bound: self.bound, - _phantom: PhantomData, - } + *self } } diff --git a/quickwit/quickwit-common/src/tower/pool.rs b/quickwit/quickwit-common/src/tower/pool.rs index c4029fb400d..c26c2ea07b9 100644 --- a/quickwit/quickwit-common/src/tower/pool.rs +++ b/quickwit/quickwit-common/src/tower/pool.rs @@ -20,20 +20,19 @@ use std::any::TypeId; use std::borrow::Borrow; use std::cmp::{Eq, PartialEq}; -use std::collections::HashMap; use std::fmt; use std::hash::Hash; use std::sync::Arc; +use dashmap::DashMap; use futures::{Stream, StreamExt}; -use tokio::sync::RwLock; use super::Change; /// A pool of `V` values identified by `K` keys. The pool can be updated manually by calling the /// `add/remove` methods or by listening to a stream of changes. pub struct Pool { - inner: Arc>>, + pool: Arc>, } impl fmt::Debug for Pool @@ -49,26 +48,21 @@ where impl Clone for Pool { fn clone(&self) -> Self { Self { - inner: self.inner.clone(), + pool: self.pool.clone(), } } } -impl Default for Pool { +impl Default for Pool +where K: Eq + PartialEq + Hash +{ fn default() -> Self { - let inner = InnerPool { - map: HashMap::new(), - }; Self { - inner: Arc::new(RwLock::new(inner)), + pool: Arc::new(DashMap::default()), } } } -struct InnerPool { - map: HashMap, -} - impl Pool where K: Eq + PartialEq + Hash + Clone + Send + Sync + 'static, @@ -85,10 +79,10 @@ where .for_each(|change| async { match change { Change::Insert(key, service) => { - pool.insert(key, service).await; + pool.insert(key, service); } Change::Remove(key) => { - pool.remove(&key).await; + pool.remove(&key); } } }) @@ -98,59 +92,66 @@ where } /// Returns whether the pool is empty. - pub async fn is_empty(&self) -> bool { - self.inner.read().await.map.is_empty() + pub fn is_empty(&self) -> bool { + self.pool.is_empty() } /// Returns the number of values in the pool. - pub async fn len(&self) -> usize { - self.inner.read().await.map.len() + pub fn len(&self) -> usize { + self.pool.len() } /// Returns all the keys in the pool. - pub async fn keys(&self) -> Vec { - self.inner.read().await.map.keys().cloned().collect() + pub fn keys(&self) -> impl Iterator + '_ { + self.pool.iter().map(|entry| entry.key().clone()) + } + + /// Returns all the values in the pool. + pub fn values(&self) -> impl Iterator + '_ { + self.pool.iter().map(|entry| entry.value().clone()) } /// Returns all the key-value pairs in the pool. - pub async fn all(&self) -> Vec<(K, V)> { - self.inner - .read() - .await - .map + pub fn pairs(&self) -> impl Iterator + '_ { + self.pool .iter() - .map(|(k, v)| (k.clone(), v.clone())) - .collect() + .map(|entry| (entry.key().clone(), entry.value().clone())) } /// Returns the value associated with the given key. - pub async fn get(&self, key: &Q) -> Option + pub fn contains_key(&self, key: &Q) -> bool where Q: Hash + Eq + ?Sized, K: Borrow, { - self.inner.read().await.map.get(key).cloned() + self.pool.contains_key(key) + } + + /// Returns the value associated with the given key. + pub fn get(&self, key: &Q) -> Option + where + Q: Hash + Eq + ?Sized, + K: Borrow, + { + self.pool.get(key).map(|entry| entry.value().clone()) } /// Finds a key in the pool that satisfies the given predicate. - pub async fn find(&self, func: impl Fn(&K) -> bool) -> Option { - self.inner - .read() - .await - .map - .keys() - .find(|k| func(k)) - .cloned() + pub fn find(&self, func: impl Fn(&K) -> bool) -> Option { + self.pool + .iter() + .find(|entry| func(entry.key())) + .map(|entry| entry.key().clone()) } /// Adds a value to the pool. - pub async fn insert(&self, key: K, service: V) { - self.inner.write().await.map.insert(key, service); + pub fn insert(&self, key: K, service: V) { + self.pool.insert(key, service); } /// Removes a value from the pool. - pub async fn remove(&self, key: &K) { - self.inner.write().await.map.remove(key); + pub fn remove(&self, key: &K) { + self.pool.remove(key); } } @@ -159,11 +160,8 @@ where K: Eq + PartialEq + Hash { fn from_iter(iter: I) -> Self where I: IntoIterator { - let key_values = HashMap::from_iter(iter); - let inner = InnerPool { map: key_values }; - Self { - inner: Arc::new(RwLock::new(inner)), + pool: Arc::new(DashMap::from_iter(iter)), } } } @@ -180,39 +178,48 @@ mod tests { async fn test_pool() { let (change_stream_tx, change_stream_rx) = tokio::sync::mpsc::channel(10); let change_stream = ReceiverStream::new(change_stream_rx); + let pool = Pool::default(); pool.listen_for_changes(change_stream); - assert!(pool.is_empty().await); - assert_eq!(pool.len().await, 0); + + assert!(pool.is_empty()); + assert_eq!(pool.len(), 0); change_stream_tx.send(Change::Insert(1, 11)).await.unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert!(!pool.is_empty().await); - assert_eq!(pool.len().await, 1); - assert_eq!(pool.get(&1).await, Some(11)); + + assert!(!pool.is_empty()); + assert_eq!(pool.len(), 1); + + assert!(pool.contains_key(&1)); + assert_eq!(pool.get(&1), Some(11)); change_stream_tx.send(Change::Insert(2, 21)).await.unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert_eq!(pool.len().await, 2); - assert_eq!(pool.get(&2).await, Some(21)); - assert_eq!(pool.find(|k| *k == 1).await, Some(1)); + assert_eq!(pool.len(), 2); + assert_eq!(pool.get(&2), Some(21)); + + assert_eq!(pool.find(|k| *k == 1), Some(1)); + + let mut pairs = pool.pairs().collect::>(); + pairs.sort(); - let mut all_nodes = pool.all().await; - all_nodes.sort(); - assert_eq!(all_nodes, vec![(1, 11), (2, 21)]); + assert_eq!(pairs, vec![(1, 11), (2, 21)]); change_stream_tx.send(Change::Insert(1, 12)).await.unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert_eq!(pool.get(&1).await, Some(12)); + + assert_eq!(pool.get(&1), Some(12)); change_stream_tx.send(Change::Remove(1)).await.unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert_eq!(pool.len().await, 1); + + assert_eq!(pool.len(), 1); change_stream_tx.send(Change::Remove(2)).await.unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert!(pool.is_empty().await); - assert_eq!(pool.len().await, 0); + + assert!(pool.is_empty()); } } diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index 291884e7fdf..ed8e5cbd965 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -378,7 +378,7 @@ impl IndexConfig { default_search_fields: vec![ "body".to_string(), r#"attributes.server"#.to_string(), - r#"attributes.server\.status"#.to_string(), + r"attributes.server\.status".to_string(), ], }; IndexConfig { diff --git a/quickwit/quickwit-control-plane/Cargo.toml b/quickwit/quickwit-control-plane/Cargo.toml index 134d2fc333d..821a8fd5e1b 100644 --- a/quickwit/quickwit-control-plane/Cargo.toml +++ b/quickwit/quickwit-control-plane/Cargo.toml @@ -12,6 +12,7 @@ documentation = "https://quickwit.io/docs/" [dependencies] anyhow = { workspace = true } async-trait = { workspace = true } +fnv = { workspace = true } dyn-clone = { workspace = true } http = { workspace = true } hyper = { workspace = true } diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index f2412616a23..3ea7949f87d 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -28,17 +28,16 @@ use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_ingest::IngesterPool; use quickwit_metastore::Metastore; use quickwit_proto::control_plane::{ - CloseShardsRequest, CloseShardsResponse, ControlPlaneError, ControlPlaneResult, - GetOpenShardsRequest, GetOpenShardsResponse, NotifyIndexChangeRequest, - NotifyIndexChangeResponse, + ControlPlaneError, ControlPlaneResult, GetOrCreateOpenShardsRequest, + GetOrCreateOpenShardsResponse, NotifyIndexChangeRequest, NotifyIndexChangeResponse, }; use quickwit_proto::metastore::events::{ AddSourceEvent, CreateIndexEvent, DeleteIndexEvent, DeleteSourceEvent, ToggleSourceEvent, }; use quickwit_proto::metastore::{ - serde_utils as metastore_serde_utils, AddSourceRequest, CreateIndexRequest, - CreateIndexResponse, DeleteIndexRequest, DeleteSourceRequest, EmptyResponse, - ToggleSourceRequest, + serde_utils as metastore_serde_utils, AddSourceRequest, CloseShardsRequest, CreateIndexRequest, + CreateIndexResponse, DeleteIndexRequest, DeleteShardsRequest, DeleteSourceRequest, + EmptyResponse, ToggleSourceRequest, }; use quickwit_proto::{IndexUid, NodeId}; use tracing::debug; @@ -299,12 +298,12 @@ impl Handler for ControlPlane { } #[async_trait] -impl Handler for ControlPlane { - type Reply = ControlPlaneResult; +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; async fn handle( &mut self, - request: GetOpenShardsRequest, + request: GetOrCreateOpenShardsRequest, _: &ActorContext, ) -> Result { let response = handle_ask_res!(self.ingest_controller_mailbox.ask_for_res(request).await); @@ -314,7 +313,7 @@ impl Handler for ControlPlane { #[async_trait] impl Handler for ControlPlane { - type Reply = ControlPlaneResult; + type Reply = ControlPlaneResult; async fn handle( &mut self, @@ -326,11 +325,25 @@ impl Handler for ControlPlane { } } +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + request: DeleteShardsRequest, + _: &ActorContext, + ) -> Result { + let response = handle_ask_res!(self.ingest_controller_mailbox.ask_for_res(request).await); + Ok(Ok(response)) + } +} + #[cfg(test)] mod tests { use quickwit_config::{SourceParams, INGEST_SOURCE_ID}; use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::control_plane::GetOpenShardsSubrequest; + use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; use quickwit_proto::ingest::Shard; use quickwit_proto::metastore::{ListShardsResponse, ListShardsSubresponse, SourceType}; @@ -620,10 +633,11 @@ mod tests { metastore, replication_factor, ); - let get_open_shards_request = GetOpenShardsRequest { - subrequests: vec![GetOpenShardsSubrequest { + let get_open_shards_request = GetOrCreateOpenShardsRequest { + subrequests: vec![GetOrCreateOpenShardsSubrequest { index_id: "test-index".to_string(), source_id: INGEST_SOURCE_ID.to_string(), + closed_shards: Vec::new(), }], unavailable_ingesters: Vec::new(), }; diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 1a4afaf580d..95e49b09e12 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -19,29 +19,29 @@ use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet}; -use std::fmt; -use std::hash::{Hash, Hasher}; -use std::ops::Deref; use std::sync::Arc; use std::time::{Duration, Instant}; use anyhow::Context; use async_trait::async_trait; +use fnv::FnvHashMap; use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; use quickwit_config::INGEST_SOURCE_ID; use quickwit_ingest::IngesterPool; use quickwit_metastore::{ListIndexesQuery, Metastore}; use quickwit_proto::control_plane::{ - CloseShardsRequest, CloseShardsResponse, ControlPlaneError, ControlPlaneResult, - GetOpenShardsRequest, GetOpenShardsResponse, GetOpenShardsSubresponse, + ControlPlaneError, ControlPlaneResult, GetOpenShardsSubresponse, GetOrCreateOpenShardsRequest, + GetOrCreateOpenShardsResponse, }; use quickwit_proto::ingest::ingester::{IngesterService, PingRequest}; use quickwit_proto::ingest::{IngestV2Error, Shard, ShardState}; use quickwit_proto::metastore::events::{ AddSourceEvent, CreateIndexEvent, DeleteIndexEvent, DeleteSourceEvent, }; -use quickwit_proto::metastore::{EntityKind, MetastoreError}; +use quickwit_proto::metastore::{ + CloseShardsRequest, DeleteShardsRequest, EmptyResponse, EntityKind, MetastoreError, +}; use quickwit_proto::types::{IndexId, NodeId, SourceId}; use quickwit_proto::{metastore, IndexUid, NodeIdRef, ShardId}; use rand::seq::SliceRandom; @@ -57,64 +57,38 @@ const PING_LEADER_TIMEOUT: Duration = if cfg!(test) { type NextShardId = ShardId; -/// A wrapper around `Shard` that implements `Hash` to allow insertion of shards into a `HashSet`. -struct ShardEntry(Shard); - -impl fmt::Debug for ShardEntry { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("ShardEntry") - .field("shard_id", &self.0.shard_id) - .field("leader_id", &self.0.leader_id) - .field("follower_id", &self.0.follower_id) - .field("shard_state", &self.0.shard_state) - .finish() - } -} - -impl Deref for ShardEntry { - type Target = Shard; - - fn deref(&self) -> &Self::Target { - &self.0 - } -} - -impl Hash for ShardEntry { - fn hash(&self, state: &mut H) { - self.0.shard_id.hash(state); - } -} - -impl PartialEq for ShardEntry { - fn eq(&self, other: &Self) -> bool { - self.0.shard_id == other.0.shard_id - } -} - -impl Eq for ShardEntry {} - #[derive(Debug, Eq, PartialEq)] struct ShardTableEntry { - shard_entries: HashSet, + shards: FnvHashMap, next_shard_id: NextShardId, } impl Default for ShardTableEntry { fn default() -> Self { Self { - shard_entries: HashSet::new(), - next_shard_id: 1, // `1` matches the PostgreSQL sequence min value. + shards: Default::default(), + next_shard_id: Self::DEFAULT_NEXT_SHARD_ID, } } } impl ShardTableEntry { + const DEFAULT_NEXT_SHARD_ID: NextShardId = 1; // `1` matches the PostgreSQL sequence min value. + + fn is_empty(&self) -> bool { + self.shards.is_empty() + } + + fn is_default(&self) -> bool { + self.is_empty() && self.next_shard_id == Self::DEFAULT_NEXT_SHARD_ID + } + #[cfg(test)] - fn shards(&self) -> Vec { - self.shard_entries - .iter() - .map(|shard_entry| shard_entry.0.clone()) - .sorted_by_key(|shard| shard.shard_id) + fn sorted_shards(&self) -> Vec { + self.shards + .values() + .cloned() + .sorted_unstable_by_key(|shard| shard.shard_id) .collect() } } @@ -133,7 +107,7 @@ impl ShardTable { let previous_table_entry_opt = self.table_entries.insert(key, table_entry); if let Some(previous_table_entry) = previous_table_entry_opt { - if previous_table_entry != ShardTableEntry::default() { + if !previous_table_entry.is_default() { error!( "shard table entry for index `{}` and source `{}` already exists", index_uid.index_id(), @@ -159,13 +133,13 @@ impl ShardTable { let key = (index_uid.clone(), source_id.clone()); let table_entry = self.table_entries.get(&key)?; let open_shards: Vec = table_entry - .shard_entries - .iter() - .filter(|shard_entry| { - shard_entry.is_open() - && !unavailable_ingesters.contains(NodeIdRef::from_str(&shard_entry.leader_id)) + .shards + .values() + .filter(|shard| { + shard.is_open() + && !unavailable_ingesters.contains(NodeIdRef::from_str(&shard.leader_id)) }) - .map(|shard_entry| shard_entry.0.clone()) + .cloned() .collect(); #[cfg(test)] @@ -190,8 +164,7 @@ impl ShardTable { Entry::Occupied(mut entry) => { for shard in shards { let table_entry = entry.get_mut(); - let shard_entry = ShardEntry(shard.clone()); - table_entry.shard_entries.replace(shard_entry); + table_entry.shards.insert(shard.shard_id, shard.clone()); table_entry.next_shard_id = next_shard_id; } } @@ -199,10 +172,13 @@ impl ShardTable { // the metastore, so should we panic here? Warnings are most likely going to go // unnoticed. Entry::Vacant(entry) => { - let shard_entries: HashSet = - shards.iter().cloned().map(ShardEntry).collect(); + let shards: FnvHashMap = shards + .iter() + .cloned() + .map(|shard| (shard.shard_id, shard)) + .collect(); let table_entry = ShardTableEntry { - shard_entries, + shards, next_shard_id, }; entry.insert(table_entry); @@ -210,24 +186,38 @@ impl ShardTable { } } - /// Evicts the shards identified by their index UID, source ID, and shard IDs from the shard - /// table. + /// Sets the state of the shards identified by their index UID, source ID, and shard IDs to + /// `Closed`. + fn close_shards(&mut self, index_uid: &IndexUid, source_id: &SourceId, shard_ids: &[ShardId]) { + let key = (index_uid.clone(), source_id.clone()); + + if let Some(table_entry) = self.table_entries.get_mut(&key) { + for shard_id in shard_ids { + if let Some(shard) = table_entry.shards.get_mut(shard_id) { + shard.shard_state = ShardState::Closed as i32; + } + } + } + } + + /// Removes the shards identified by their index UID, source ID, and shard IDs. fn remove_shards(&mut self, index_uid: &IndexUid, source_id: &SourceId, shard_ids: &[ShardId]) { let key = (index_uid.clone(), source_id.clone()); if let Some(table_entry) = self.table_entries.get_mut(&key) { - table_entry - .shard_entries - .retain(|shard_entry| !shard_ids.contains(&shard_entry.shard_id)); + for shard_id in shard_ids { + table_entry.shards.remove(shard_id); + } } } - /// Removes all the entries that match the target index ID. + /// Removes all entries that match the target index ID. fn remove_index(&mut self, index_id: &str) { self.table_entries .retain(|(index_uid, _), _| index_uid.index_id() != index_id); } + /// Removes any entry that matches the target index UID and source ID. fn remove_source(&mut self, index_uid: &IndexUid, source_id: &SourceId) { let key = (index_uid.clone(), source_id.clone()); self.table_entries.remove(&key); @@ -309,13 +299,13 @@ impl IngestController { list_shards_subresponse.index_uid.into(), list_shards_subresponse.source_id, ); - let shard_entries: HashSet = list_shards_subresponse + let shards: FnvHashMap = list_shards_subresponse .shards .into_iter() - .map(ShardEntry) + .map(|shard| (shard.shard_id, shard)) .collect(); let table_entry = ShardTableEntry { - shard_entries, + shards, next_shard_id: list_shards_subresponse.next_shard_id, }; self.shard_table.table_entries.insert(key, table_entry); @@ -344,7 +334,6 @@ impl IngestController { let mut leader_ingester = self .ingester_pool .get(leader_id) - .await .ok_or(PingError::LeaderUnavailable)?; let ping_request = PingRequest { @@ -377,8 +366,11 @@ impl IngestController { ctx: &ActorContext, unavailable_ingesters: &mut HashSet, ) -> Option<(NodeId, Option)> { - let mut candidates: Vec = self.ingester_pool.keys().await; - candidates.retain(|node_id| !unavailable_ingesters.contains(node_id)); + let mut candidates: Vec = self + .ingester_pool + .keys() + .filter(|node_id| !unavailable_ingesters.contains(node_id)) + .collect(); candidates.shuffle(&mut rand::thread_rng()); #[cfg(test)] @@ -426,15 +418,15 @@ impl IngestController { None } - /// Finds the open shards that satisfies the [`GetOpenShardsRequest`] request sent by an + /// Finds the open shards that satisfies the [`GetOrCreateOpenShardsRequest`] request sent by an /// ingest router. First, the control plane checks its internal shard table to find /// candidates. If it does not contain any, the control plane will ask /// the metastore to open new shards. async fn get_open_shards( &mut self, ctx: &ActorContext, - get_open_shards_request: GetOpenShardsRequest, - ) -> ControlPlaneResult { + get_open_shards_request: GetOrCreateOpenShardsRequest, + ) -> ControlPlaneResult { let mut get_open_shards_subresponses = Vec::with_capacity(get_open_shards_request.subrequests.len()); @@ -456,6 +448,13 @@ impl IngestController { index_id: get_open_shards_subrequest.index_id.clone(), }) })?; + // if !get_open_shards_subrequest.closed_shards.is_empty() { + // self.shard_table.delete_shards( + // &index_uid, + // &get_open_shards_subrequest.source_id, + // &get_open_shards_subrequest.closed_shards, + // ); + // } let (open_shards, next_shard_id) = self .shard_table .find_open_shards( @@ -520,7 +519,7 @@ impl IngestController { get_open_shards_subresponses.push(get_open_shards_subresponse); } } - let get_open_shards_response = GetOpenShardsResponse { + let get_open_shards_response = GetOrCreateOpenShardsResponse { subresponses: get_open_shards_subresponses, }; Ok(get_open_shards_response) @@ -528,38 +527,36 @@ impl IngestController { async fn close_shards( &mut self, - ctx: &ActorContext, + _ctx: &ActorContext, close_shards_request: CloseShardsRequest, - ) -> ControlPlaneResult { - let mut close_shards_subrequests = - Vec::with_capacity(close_shards_request.subrequests.len()); + ) -> ControlPlaneResult { for close_shards_subrequest in close_shards_request.subrequests { - let close_shards_subrequest = metastore::CloseShardsSubrequest { - index_uid: close_shards_subrequest.index_uid, - source_id: close_shards_subrequest.source_id, - shard_id: close_shards_subrequest.shard_id, - shard_state: close_shards_subrequest.shard_state, - replication_position_inclusive: close_shards_subrequest - .replication_position_inclusive, - }; - close_shards_subrequests.push(close_shards_subrequest); + let index_uid: IndexUid = close_shards_subrequest.index_uid.into(); + let source_id = close_shards_subrequest.source_id; + // TODO: Group by (index_uid, source_id) first, or change schema of + // `CloseShardsSubrequest`. + let shard_ids = vec![close_shards_subrequest.shard_id]; + + self.shard_table + .close_shards(&index_uid, &source_id, &shard_ids) } - let metastore_close_shards_request = metastore::CloseShardsRequest { - subrequests: close_shards_subrequests, - }; - let close_shards_response = ctx - .protect_future(self.metastore.close_shards(metastore_close_shards_request)) - .await?; - for close_shards_success in close_shards_response.successes { - let index_uid: IndexUid = close_shards_success.index_uid.into(); - self.shard_table.remove_shards( - &index_uid, - &close_shards_success.source_id, - &[close_shards_success.shard_id], - ); + Ok(EmptyResponse {}) + } + + async fn delete_shards( + &mut self, + _ctx: &ActorContext, + delete_shards_request: DeleteShardsRequest, + ) -> ControlPlaneResult { + for delete_shards_subrequest in delete_shards_request.subrequests { + let index_uid: IndexUid = delete_shards_subrequest.index_uid.into(); + let source_id = delete_shards_subrequest.source_id; + let shard_ids = delete_shards_subrequest.shard_ids; + + self.shard_table + .remove_shards(&index_uid, &source_id, &shard_ids) } - let close_shards_response = CloseShardsResponse {}; - Ok(close_shards_response) + Ok(EmptyResponse {}) } } @@ -648,6 +645,7 @@ impl Handler for IngestController { event: DeleteSourceEvent, _ctx: &ActorContext, ) -> Result { + // TODO: We need to let the routers and ingesters know. self.shard_table .remove_source(&event.index_uid, &event.source_id); Ok(Ok(())) @@ -655,12 +653,12 @@ impl Handler for IngestController { } #[async_trait] -impl Handler for IngestController { - type Reply = ControlPlaneResult; +impl Handler for IngestController { + type Reply = ControlPlaneResult; async fn handle( &mut self, - request: GetOpenShardsRequest, + request: GetOrCreateOpenShardsRequest, ctx: &ActorContext, ) -> Result { let response_res = self.get_open_shards(ctx, request).await; @@ -670,7 +668,7 @@ impl Handler for IngestController { #[async_trait] impl Handler for IngestController { - type Reply = ControlPlaneResult; + type Reply = ControlPlaneResult; async fn handle( &mut self, @@ -682,13 +680,27 @@ impl Handler for IngestController { } } +#[async_trait] +impl Handler for IngestController { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + request: DeleteShardsRequest, + ctx: &ActorContext, + ) -> Result { + let response_res = self.delete_shards(ctx, request).await; + Ok(response_res) + } +} + #[cfg(test)] mod tests { use quickwit_actors::Universe; use quickwit_config::SourceConfig; use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::control_plane::GetOpenShardsSubrequest; + use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; use quickwit_proto::ingest::ingester::{ IngesterServiceClient, MockIngesterService, PingResponse, }; @@ -709,7 +721,7 @@ mod tests { let key = (index_uid.clone(), source_id.clone()); let table_entry = shard_table.table_entries.get(&key).unwrap(); - assert!(table_entry.shard_entries.is_empty()); + assert!(table_entry.shards.is_empty()); assert_eq!(table_entry.next_shard_id, 1); } @@ -806,7 +818,7 @@ mod tests { let key = (index_uid_0.clone(), source_id.clone()); let table_entry = shard_table.table_entries.get(&key).unwrap(); - let shards = table_entry.shards(); + let shards = table_entry.sorted_shards(); assert_eq!(shards.len(), 1); assert_eq!(shards[0], shard_01); assert_eq!(table_entry.next_shard_id, 2); @@ -833,7 +845,7 @@ mod tests { let key = (index_uid_0.clone(), source_id.clone()); let table_entry = shard_table.table_entries.get(&key).unwrap(); - let shards = table_entry.shards(); + let shards = table_entry.sorted_shards(); assert_eq!(shards.len(), 2); assert_eq!(shards[0], shard_01); assert_eq!(shards[1], shard_02); @@ -841,7 +853,7 @@ mod tests { } #[test] - fn test_shard_table_remove_shards() { + fn test_shard_table_delete_shards() { let index_uid_0: IndexUid = "test-index:0".into(); let index_uid_1: IndexUid = "test-index:1".into(); let source_id = "test-source".to_string(); @@ -881,14 +893,14 @@ mod tests { let key = (index_uid_0.clone(), source_id.clone()); let table_entry = shard_table.table_entries.get(&key).unwrap(); - let shards = table_entry.shards(); + let shards = table_entry.sorted_shards(); assert_eq!(shards.len(), 1); assert_eq!(shards[0], shard_01); assert_eq!(table_entry.next_shard_id, 3); let key = (index_uid_1.clone(), source_id.clone()); let table_entry = shard_table.table_entries.get(&key).unwrap(); - assert_eq!(table_entry.shard_entries.len(), 0); + assert_eq!(table_entry.shards.len(), 0); assert_eq!(table_entry.next_shard_id, 2); } @@ -957,11 +969,11 @@ mod tests { assert_eq!(ingest_controller.index_table.len(), 2); assert_eq!( ingest_controller.index_table["test-index-0"], - "test-index-0:0".into() + "test-index-0:0" ); assert_eq!( ingest_controller.index_table["test-index-1"], - "test-index-1:0".into() + "test-index-1:0" ); assert_eq!(ingest_controller.shard_table.table_entries.len(), 2); @@ -972,7 +984,7 @@ mod tests { .table_entries .get(&key) .unwrap(); - let shards = table_entry.shards(); + let shards = table_entry.sorted_shards(); assert_eq!(shards.len(), 1); assert_eq!(shards[0].shard_id, 42); assert_eq!(table_entry.next_shard_id, 43); @@ -983,7 +995,7 @@ mod tests { .table_entries .get(&key) .unwrap(); - let shards = table_entry.shards(); + let shards = table_entry.sorted_shards(); assert_eq!(shards.len(), 0); assert_eq!(table_entry.next_shard_id, 1); } @@ -1017,9 +1029,7 @@ mod tests { Ok(PingResponse {}) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); ingest_controller .ping_leader_and_follower(&ctx, &leader_id, None) @@ -1037,9 +1047,7 @@ mod tests { }) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); let error = ingest_controller .ping_leader_and_follower(&ctx, &leader_id, None) @@ -1058,9 +1066,7 @@ mod tests { }) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); let follower_id: NodeId = "test-ingester-1".into(); let error = ingest_controller @@ -1097,9 +1103,7 @@ mod tests { Err(IngestV2Error::Internal("Io error".to_string())) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); let leader_follower_pair = ingest_controller .find_leader_and_follower(&ctx, &mut HashSet::new()) @@ -1114,9 +1118,7 @@ mod tests { Ok(PingResponse {}) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-1".into(), ingester) - .await; + ingester_pool.insert("test-ingester-1".into(), ingester); let (leader_id, follower_id) = ingest_controller .find_leader_and_follower(&ctx, &mut HashSet::new()) @@ -1155,18 +1157,14 @@ mod tests { }) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); let mut mock_ingester = MockIngesterService::default(); mock_ingester.expect_ping().returning(|_request| { panic!("`test-ingester-1` should not be pinged."); }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-1".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-1".into(), ingester.clone()); let leader_follower_pair = ingest_controller .find_leader_and_follower(&ctx, &mut HashSet::new()) @@ -1189,18 +1187,14 @@ mod tests { Ok(PingResponse {}) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester.clone()); let mut mock_ingester = MockIngesterService::default(); mock_ingester.expect_ping().returning(|_request| { panic!("`test-ingester-2` should not be pinged."); }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-2".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-2".into(), ingester.clone()); let (leader_id, follower_id) = ingest_controller .find_leader_and_follower(&ctx, &mut HashSet::new()) @@ -1258,15 +1252,11 @@ mod tests { Ok(PingResponse {}) }); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-1".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-1".into(), ingester.clone()); let mock_ingester = MockIngesterService::default(); let ingester: IngesterServiceClient = mock_ingester.into(); - ingester_pool - .insert("test-ingester-2".into(), ingester.clone()) - .await; + ingester_pool.insert("test-ingester-2".into(), ingester.clone()); let replication_factor = 2; let mut ingest_controller = @@ -1303,7 +1293,7 @@ mod tests { .shard_table .update_shards(&index_uid_0, &source_id, &shards, 3); - let request = GetOpenShardsRequest { + let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), unavailable_ingesters: Vec::new(), }; @@ -1314,17 +1304,19 @@ mod tests { assert_eq!(response.subresponses.len(), 0); let subrequests = vec![ - GetOpenShardsSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-0".to_string(), source_id: source_id.clone(), + closed_shards: Vec::new(), }, - GetOpenShardsSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-1".to_string(), source_id: source_id.clone(), + closed_shards: Vec::new(), }, ]; let unavailable_ingesters = vec!["test-ingester-0".to_string()]; - let request = GetOpenShardsRequest { + let request = GetOrCreateOpenShardsRequest { subrequests, unavailable_ingesters, }; diff --git a/quickwit/quickwit-control-plane/src/lib.rs b/quickwit/quickwit-control-plane/src/lib.rs index 650204be433..8fbe126aca2 100644 --- a/quickwit/quickwit-control-plane/src/lib.rs +++ b/quickwit/quickwit-control-plane/src/lib.rs @@ -32,7 +32,7 @@ use quickwit_proto::indexing::{IndexingServiceClient, IndexingTask}; use quickwit_proto::metastore::events::{ AddSourceEvent, CreateIndexEvent, DeleteIndexEvent, DeleteSourceEvent, ToggleSourceEvent, }; -use quickwit_proto::metastore::SourceType; +use quickwit_proto::metastore::{CloseShardsRequest, DeleteShardsRequest, SourceType}; use tracing::error; /// Indexer-node specific information stored in the pool of available indexer nodes @@ -112,6 +112,30 @@ impl EventSubscriber for ControlPlaneEventSubscriber { } } +#[async_trait] +impl EventSubscriber for ControlPlaneEventSubscriber { + async fn handle_event(&mut self, request: CloseShardsRequest) { + if let Err(error) = self.0.close_shards(request).await { + error!( + "failed to notify control plane of close shards event: `{}`", + error + ); + } + } +} + +#[async_trait] +impl EventSubscriber for ControlPlaneEventSubscriber { + async fn handle_event(&mut self, request: DeleteShardsRequest) { + if let Err(error) = self.0.delete_shards(request).await { + error!( + "failed to notify control plane of delete shards event: `{}`", + error + ); + } + } +} + #[cfg(test)] mod tests { diff --git a/quickwit/quickwit-control-plane/src/scheduler.rs b/quickwit/quickwit-control-plane/src/scheduler.rs index 5b519217ec9..f881cfd5988 100644 --- a/quickwit/quickwit-control-plane/src/scheduler.rs +++ b/quickwit/quickwit-control-plane/src/scheduler.rs @@ -168,7 +168,7 @@ impl IndexingScheduler { } async fn schedule_indexing_plan_if_needed(&mut self) -> anyhow::Result<()> { - let mut indexers = self.get_indexers_from_indexer_pool().await; + let mut indexers = self.get_indexers_from_indexer_pool(); if indexers.is_empty() { warn!("No indexer available, cannot schedule an indexing plan."); return Ok(()); @@ -303,7 +303,7 @@ impl IndexingScheduler { } } - let mut indexers = self.get_indexers_from_indexer_pool().await; + let mut indexers = self.get_indexers_from_indexer_pool(); let running_indexing_tasks_by_node_id: HashMap> = indexers .iter() .map(|indexer| (indexer.0.clone(), indexer.1.indexing_tasks.clone())) @@ -325,8 +325,8 @@ impl IndexingScheduler { Ok(()) } - async fn get_indexers_from_indexer_pool(&self) -> Vec<(String, IndexerNodeInfo)> { - self.indexer_pool.all().await + fn get_indexers_from_indexer_pool(&self) -> Vec<(String, IndexerNodeInfo)> { + self.indexer_pool.pairs().collect() } async fn apply_physical_indexing_plan( diff --git a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs index f0802ceb1cc..9a84a03b099 100644 --- a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs +++ b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/default_mapper.rs @@ -651,7 +651,7 @@ mod tests { default_search_field_names.sort(); assert_eq!( default_search_field_names, - ["attributes.server", r#"attributes.server\.status"#, "body"] + ["attributes.server", r"attributes.server\.status", "body"] ); assert_eq!(config.field_mappings.num_fields(), 10); Ok(()) @@ -1639,7 +1639,7 @@ mod tests { r#"TermQuery(Term(field=2, type=Str, "toto"))"# ); assert_eq!( - default_doc_mapper_query_aux(&doc_mapper, r#"identity\.username:toto"#).unwrap(), + default_doc_mapper_query_aux(&doc_mapper, r"identity\.username:toto").unwrap(), r#"TermQuery(Term(field=3, type=Str, "toto"))"# ); } @@ -1660,7 +1660,7 @@ mod tests { r#"TermQuery(Term(field=2, type=Json, path=username, type=Str, "toto"))"# ); assert_eq!( - default_doc_mapper_query_aux(&doc_mapper, r#"identity\.username:toto"#).unwrap(), + default_doc_mapper_query_aux(&doc_mapper, r"identity\.username:toto").unwrap(), r#"TermQuery(Term(field=3, type=Str, "toto"))"# ); } diff --git a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs index 8eaa22cc9d9..fcaf4323a31 100644 --- a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs +++ b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mapping_tree.rs @@ -815,13 +815,10 @@ mod tests { super::field_name_for_field_path(&["one", "two", "three"]), "one.two.three" ); - assert_eq!( - super::field_name_for_field_path(&["one.two"]), - r#"one\.two"# - ); + assert_eq!(super::field_name_for_field_path(&["one.two"]), r"one\.two"); assert_eq!( super::field_name_for_field_path(&["one.two", "three"]), - r#"one\.two.three"# + r"one\.two.three" ); } @@ -1353,11 +1350,11 @@ mod tests { vec!["one", "two", "three"] ); assert_eq!( - super::build_field_path_from_str(r#"one\.two"#), + super::build_field_path_from_str(r"one\.two"), vec!["one.two"] ); assert_eq!( - super::build_field_path_from_str(r#"one\.two.three"#), + super::build_field_path_from_str(r"one\.two.three"), vec!["one.two", "three"] ); assert_eq!(super::build_field_path_from_str(r#"one."#), vec!["one"]); diff --git a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mod.rs b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mod.rs index 42beb6a6912..73eb7b97263 100644 --- a/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mod.rs +++ b/quickwit/quickwit-doc-mapper/src/default_doc_mapper/mod.rs @@ -46,7 +46,7 @@ pub(crate) use self::tokenizer_entry::{ use crate::QW_RESERVED_FIELD_NAMES; /// Regular expression validating a field mapping name. -pub const FIELD_MAPPING_NAME_PATTERN: &str = r#"^[_\-a-zA-Z][_\.\-a-zA-Z0-9]{0,254}$"#; +pub const FIELD_MAPPING_NAME_PATTERN: &str = r"^[_\-a-zA-Z][_\.\-a-zA-Z0-9]{0,254}$"; /// Validates a field mapping name. /// Returns `Ok(())` if the name can be used for a field mapping. Does not check for reserved field diff --git a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs index 197ad7cad1b..5c50f1f59a2 100644 --- a/quickwit/quickwit-doc-mapper/src/doc_mapper.rs +++ b/quickwit/quickwit-doc-mapper/src/doc_mapper.rs @@ -197,11 +197,9 @@ impl WarmupInfo { /// Merge other WarmupInfo into self. pub fn merge(&mut self, other: WarmupInfo) { self.term_dict_field_names - .extend(other.term_dict_field_names.into_iter()); - self.posting_field_names - .extend(other.posting_field_names.into_iter()); - self.fast_field_names - .extend(other.fast_field_names.into_iter()); + .extend(other.term_dict_field_names); + self.posting_field_names.extend(other.posting_field_names); + self.fast_field_names.extend(other.fast_field_names); self.field_norms |= other.field_norms; for (field, term_and_pos) in other.terms_grouped_by_field.into_iter() { diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 7d06d3982ce..8f91f70949a 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -57,12 +57,12 @@ pub enum IndexServiceError { } impl ServiceError for IndexServiceError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { Self::Internal(_) => ServiceErrorCode::Internal, Self::InvalidConfig(_) => ServiceErrorCode::BadRequest, Self::InvalidIdentifier(_) => ServiceErrorCode::BadRequest, - Self::Metastore(error) => error.status_code(), + Self::Metastore(error) => error.error_code(), Self::OperationNotAllowed(_) => ServiceErrorCode::MethodNotAllowed, Self::SplitDeletion(_) => ServiceErrorCode::Internal, Self::Storage(_) => ServiceErrorCode::Internal, diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 8b9a3592017..c9b0b13db0f 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -631,7 +631,7 @@ mod tests { metastore .expect_stage_splits() .withf(|index_uid, _metadata| -> bool { - index_uid.to_string() == "test-index:11111111111111111111111111" + *index_uid == "test-index:11111111111111111111111111" }) .returning(|_, _| Ok(())); metastore @@ -644,7 +644,7 @@ mod tests { _publish_token_opt| -> bool { let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - index_uid.to_string() == "test-index:11111111111111111111111111" + *index_uid == "test-index:11111111111111111111111111" && checkpoint_delta.source_id == "test-source" && splits.len() == 1 && replaced_splits.is_empty() @@ -732,9 +732,7 @@ mod tests { }); metastore .expect_stage_splits() - .withf(|index_uid, _metadata| { - index_uid.to_string() == "test-index:11111111111111111111111111" - }) + .withf(|index_uid, _metadata| *index_uid == "test-index:11111111111111111111111111") .returning(|_, _| Ok(())); metastore .expect_publish_splits() @@ -746,7 +744,7 @@ mod tests { _publish_token_opt| -> bool { let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - index_uid.to_string() == "test-index:11111111111111111111111111" + *index_uid == "test-index:11111111111111111111111111" && splits.len() == 1 && replaced_split_ids.is_empty() && checkpoint_delta.source_id == "test-source" @@ -930,7 +928,7 @@ mod tests { _publish_token_opt| -> bool { let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - index_uid.to_string() == "test-index:11111111111111111111111111" + *index_uid == "test-index:11111111111111111111111111" && splits.is_empty() && replaced_split_ids.is_empty() && checkpoint_delta.source_id == "test-source" diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 697b41631cd..72ba3a65a7e 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -212,7 +212,7 @@ mod tests { checkpoint_delta_opt, _publish_token_opt| { let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - index_uid.to_string() == "index:11111111111111111111111111" + *index_uid == "index:11111111111111111111111111" && checkpoint_delta.source_id == "source" && split_ids[..] == ["split"] && replaced_split_ids.is_empty() @@ -289,7 +289,7 @@ mod tests { checkpoint_delta_opt, _publish_token_opt| { let checkpoint_delta = checkpoint_delta_opt.as_ref().unwrap(); - index_uid.to_string() == "index:11111111111111111111111111" + *index_uid == "index:11111111111111111111111111" && checkpoint_delta.source_id == "source" && split_ids.is_empty() && replaced_split_ids.is_empty() @@ -363,7 +363,7 @@ mod tests { replaced_split_ids, checkpoint_delta_opt, _publish_token_opt| { - index_uid.to_string() == "index:11111111111111111111111111" + *index_uid == "index:11111111111111111111111111" && new_split_ids[..] == ["split3"] && replaced_split_ids[..] == ["split1", "split2"] && checkpoint_delta_opt.is_none() diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index ab7d7629e5f..dff910ea6cd 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -623,7 +623,7 @@ mod tests { .expect_stage_splits() .withf(move |index_uid, metadata_list| -> bool { let is_metadata_valid = metadata_list.iter().all(|metadata| { - vec!["test-split-1", "test-split-2"].contains(&metadata.split_id()) + ["test-split-1", "test-split-2"].contains(&metadata.split_id()) && metadata.time_range == Some(1628203589..=1628203640) }); diff --git a/quickwit/quickwit-indexing/src/merge_policy/stable_log_merge_policy.rs b/quickwit/quickwit-indexing/src/merge_policy/stable_log_merge_policy.rs index 6e3c8bc29c2..b5e6c5482f2 100644 --- a/quickwit/quickwit-indexing/src/merge_policy/stable_log_merge_policy.rs +++ b/quickwit/quickwit-indexing/src/merge_policy/stable_log_merge_policy.rs @@ -235,6 +235,7 @@ impl StableLogMergePolicy { let mut current_level_max_docs = (splits[0].num_docs * 3).max(self.config.min_level_num_docs); + #[allow(clippy::single_range_in_vec_init)] let mut levels = vec![(0..current_level_max_docs)]; // for logging only for (split_ord, split) in splits.iter().enumerate() { if split.num_docs >= current_level_max_docs { diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index ced5eb00da4..5d6ac27aa3c 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -184,6 +184,9 @@ impl IngestSource { HashMap::new(); for (shard_id, truncate_position) in truncation_point { + if matches!(truncate_position, Position::Beginning) { + continue; + } let Some(leader_id) = self .assigned_shards .get(shard_id) @@ -211,7 +214,7 @@ impl IngestSource { .push(truncate_subrequest); } for (leader_id, truncate_subrequests) in per_leader_truncate_subrequests { - let Some(mut ingester) = self.ingester_pool.get(leader_id).await else { + let Some(mut ingester) = self.ingester_pool.get(leader_id) else { warn!( "failed to truncate shard: ingester `{}` is unavailable", leader_id @@ -521,9 +524,7 @@ mod tests { }); let ingester_0: IngesterServiceClient = ingester_mock_0.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester_0.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let runtime_args = Arc::new(SourceRuntimeArgs { pipeline_id, @@ -731,9 +732,7 @@ mod tests { Ok(TruncateResponse {}) }); let ingester_0: IngesterServiceClient = ingester_mock_0.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester_0.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let mut ingester_mock_1 = IngesterServiceClient::mock(); ingester_mock_1 @@ -750,9 +749,7 @@ mod tests { Ok(TruncateResponse {}) }); let ingester_1: IngesterServiceClient = ingester_mock_1.into(); - ingester_pool - .insert("test-ingester-1".into(), ingester_1.clone()) - .await; + ingester_pool.insert("test-ingester-1".into(), ingester_1.clone()); let runtime_args = Arc::new(SourceRuntimeArgs { pipeline_id, diff --git a/quickwit/quickwit-ingest/src/errors.rs b/quickwit/quickwit-ingest/src/errors.rs index 0ecfc7eb02d..40a1aeceb09 100644 --- a/quickwit/quickwit-ingest/src/errors.rs +++ b/quickwit/quickwit-ingest/src/errors.rs @@ -91,7 +91,7 @@ impl From for IngestServiceError { } impl ServiceError for IngestServiceError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { IngestServiceError::Corruption(_) => ServiceErrorCode::Internal, IngestServiceError::IndexAlreadyExists { .. } => ServiceErrorCode::BadRequest, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index 8976f2d16c9..ea8ec4f42a8 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -35,7 +35,7 @@ use tokio::sync::{mpsc, watch, RwLock}; use tokio::task::JoinHandle; use tracing::{debug, error, warn}; -use super::ingester::ShardStatus; +use super::models::ShardStatus; use crate::{ClientId, DocBatchBuilderV2, IngesterPool}; /// A fetch task is responsible for waiting and pushing new records written to a shard's record log @@ -270,7 +270,7 @@ impl MultiFetchStream { // Obtain a fetch stream from the preferred or failover ingester. let fetch_stream = loop { - let Some(mut ingester) = self.ingester_pool.get(&preferred_ingester_id).await else { + let Some(mut ingester) = self.ingester_pool.get(&preferred_ingester_id) else { if let Some(failover_ingester_id) = failover_ingester_id.take() { warn!( client_id=%self.client_id, @@ -449,7 +449,6 @@ async fn fault_tolerant_fetch_task( ); let mut ingester = ingester_pool .get(&preferred_ingester_id) - .await .expect("TODO: handle error"); let open_fetch_stream_request = OpenFetchStreamRequest { client_id: client_id.clone(), diff --git a/quickwit/quickwit-ingest/src/ingest_v2/gc.rs b/quickwit/quickwit-ingest/src/ingest_v2/gc.rs new file mode 100644 index 00000000000..0a266f625ce --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/gc.rs @@ -0,0 +1,225 @@ +// 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 std::sync::Arc; +use std::time::Duration; + +use mrecordlog::error::DeleteQueueError; +use mrecordlog::MultiRecordLog; +use once_cell::sync::Lazy; +use quickwit_proto::ingest::IngestV2Result; +use quickwit_proto::metastore::{DeleteShardsRequest, DeleteShardsSubrequest}; +use quickwit_proto::{split_queue_id, IndexUid, QueueId, ShardId, SourceId}; +use tokio::sync::{RwLock, Semaphore}; +use tracing::{error, info}; + +use super::ingest_metastore::IngestMetastore; +use super::ingester::IngesterState; + +/// Period of time after which shards candidate for deletion are actually deleted. +/// +/// A shard is candidate for deletion when it is closed, its publish position greater or equal than +/// its replication position, and the grace period has elapsed. +const GRACE_PERIOD: Duration = if cfg!(any(test, feature = "testsuite")) { + Duration::from_millis(10) +} else { + Duration::from_secs(60 * 10) +}; + +/// Maximum number of concurrent GC tasks. +const MAX_CONCURRENCY: usize = 3; + +/// Limits the number of concurrent GC tasks. +static MAX_CONCURRENCY_SEMAPHORE: Lazy> = + Lazy::new(|| Arc::new(Semaphore::new(MAX_CONCURRENCY))); + +/// Deletes shards asynchronously after the grace period has elapsed. +/// +/// 1. deletes the shards from the metastore +/// 2. removes the shards from in-memory data structures +/// 3. deletes the associated mrecordlog queues +pub(super) struct GcTask { + metastore: Arc, + mrecordlog: Arc>, + state: Arc>, + gc_candidates: Vec, +} + +impl GcTask { + pub fn spawn( + metastore: Arc, + mrecordlog: Arc>, + state: Arc>, + gc_candidates: Vec, + ) { + if gc_candidates.is_empty() { + return; + } + let gc_task = Self { + metastore, + mrecordlog, + state, + gc_candidates, + }; + tokio::spawn(gc_task.run()); + } + + async fn run(self) -> IngestV2Result<()> { + tokio::time::sleep(GRACE_PERIOD).await; + + let _permit = MAX_CONCURRENCY_SEMAPHORE + .clone() + .acquire_owned() + .await + .expect("the semaphore should be open"); + + let mut per_source_shard_ids: HashMap<(IndexUid, SourceId), Vec> = HashMap::new(); + + for gc_candidate in &self.gc_candidates { + let (index_uid, source_id, shard_id) = + split_queue_id(gc_candidate).expect("queue ID should be well-formed"); + per_source_shard_ids + .entry((index_uid, source_id)) + .or_default() + .push(shard_id); + } + let delete_shards_subrequests = per_source_shard_ids + .into_iter() + .map( + |((index_uid, source_id), shard_ids)| DeleteShardsSubrequest { + index_uid: index_uid.into(), + source_id, + shard_ids, + }, + ) + .collect(); + let delete_shards_request = DeleteShardsRequest { + subrequests: delete_shards_subrequests, + force: false, + }; + if let Err(error) = self.metastore.delete_shards(delete_shards_request).await { + error!("failed to delete shards: {}", error); + return Err(error); + } + let mut state_guard = self.state.write().await; + + for gc_candidate in &self.gc_candidates { + if state_guard.primary_shards.remove(gc_candidate).is_none() { + state_guard.replica_shards.remove(gc_candidate); + } + } + drop(state_guard); + + let mut mrecordlog_guard = self.mrecordlog.write().await; + + for gc_candidate in &self.gc_candidates { + if let Err(DeleteQueueError::IoError(error)) = + mrecordlog_guard.delete_queue(gc_candidate).await + { + error!("failed to delete mrecordlog queue: {}", error); + } + } + info!("deleted {} shard(s)", self.gc_candidates.len()); + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use quickwit_proto::ingest::ShardState; + use quickwit_proto::metastore::DeleteShardsResponse; + use quickwit_proto::queue_id; + + use super::*; + use crate::ingest_v2::ingest_metastore::MockIngestMetastore; + use crate::ingest_v2::models::{PrimaryShard, ReplicaShard}; + + #[tokio::test] + async fn test_gc_task() { + let mut mock_metastore = MockIngestMetastore::default(); + mock_metastore + .expect_delete_shards() + .once() + .returning(|request| { + assert_eq!(request.subrequests.len(), 1); + + assert_eq!(request.subrequests[0].index_uid, "test-index:0"); + assert_eq!(request.subrequests[0].source_id, "test-source"); + assert_eq!(request.subrequests[0].shard_ids, [0, 1]); + + let response = DeleteShardsResponse {}; + Ok(response) + }); + let metastore = Arc::new(mock_metastore); + + let queue_id_0 = queue_id("test-index:0", "test-source", 0); + let queue_id_1 = queue_id("test-index:0", "test-source", 1); + + let tempdir = tempfile::tempdir().unwrap(); + let mut mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); + + for queue_id in [&queue_id_0, &queue_id_1] { + mrecordlog.create_queue(queue_id).await.unwrap(); + } + let mrecordlog = Arc::new(RwLock::new(mrecordlog)); + + let mut state = IngesterState { + primary_shards: HashMap::new(), + replica_shards: HashMap::new(), + replication_clients: HashMap::new(), + replication_tasks: HashMap::new(), + }; + let primary_shard_0 = PrimaryShard::for_test( + Some("test-ingester-1"), + ShardState::Closed, + 12, + 12, + Some(12), + ); + state + .primary_shards + .insert(queue_id_0.clone(), primary_shard_0); + + let replica_shard_1 = ReplicaShard::for_test("test-ingester-1", ShardState::Closed, 42, 42); + state + .replica_shards + .insert(queue_id_1.clone(), replica_shard_1); + + let state = Arc::new(RwLock::new(state)); + + let gc_task = GcTask { + metastore, + mrecordlog: mrecordlog.clone(), + state: state.clone(), + gc_candidates: vec![queue_id_0, queue_id_1], + }; + gc_task.run().await.unwrap(); + + let state_guard = state.read().await; + assert!(state_guard.primary_shards.is_empty()); + assert!(state_guard.replica_shards.is_empty()); + + let mrecordlog_guard = mrecordlog.read().await; + assert!(mrecordlog_guard + .list_queues() + .collect::>() + .is_empty()); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs new file mode 100644 index 00000000000..570aefa595a --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs @@ -0,0 +1,43 @@ +// 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 . + +//! [`IngestMetastore`] exposes a small subset of the metastore APIs in a trait defined in this +//! crate instead of depending on the `metastore` crate directly to avoid circular dependencies. It +//! should be removed once we have migrated to the code-generated metastore client. In the meantime, +//! the concrete implementation of this trait lives in the `serve` crate. + +use async_trait::async_trait; +use quickwit_proto::ingest::IngestV2Result; +use quickwit_proto::metastore::{ + CloseShardsRequest, CloseShardsResponse, DeleteShardsRequest, DeleteShardsResponse, +}; + +#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] +#[async_trait] +pub trait IngestMetastore: Send + Sync + 'static { + async fn close_shards( + &self, + request: CloseShardsRequest, + ) -> IngestV2Result; + + async fn delete_shards( + &self, + request: DeleteShardsRequest, + ) -> IngestV2Result; +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 49e6f4b7faa..74d79c19e74 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -19,31 +19,41 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; +use std::fmt; use std::iter::once; use std::path::Path; use std::sync::Arc; use std::time::Duration; -use std::{cmp, fmt}; use async_trait::async_trait; use bytes::{Bytes, BytesMut}; use futures::stream::FuturesUnordered; use futures::StreamExt; +use mrecordlog::error::DeleteQueueError; use mrecordlog::MultiRecordLog; use quickwit_common::tower::Pool; use quickwit_common::ServiceStream; use quickwit_proto::ingest::ingester::{ AckReplicationMessage, FetchResponseV2, IngesterService, IngesterServiceClient, IngesterServiceStream, OpenFetchStreamRequest, OpenReplicationStreamRequest, - OpenReplicationStreamResponse, PersistRequest, PersistResponse, PersistSuccess, PingRequest, - PingResponse, ReplicateRequest, ReplicateSubrequest, SynReplicationMessage, TruncateRequest, - TruncateResponse, TruncateSubrequest, + OpenReplicationStreamResponse, PersistFailure, PersistFailureKind, PersistRequest, + PersistResponse, PersistSuccess, PingRequest, PingResponse, ReplicateRequest, + ReplicateSubrequest, SynReplicationMessage, TruncateRequest, TruncateResponse, + TruncateSubrequest, }; use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; +use quickwit_proto::metastore::{ + CloseShardsFailureKind, CloseShardsRequest, CloseShardsSubrequest, +}; +use quickwit_proto::split_queue_id; use quickwit_proto::types::{NodeId, QueueId}; use tokio::sync::{watch, RwLock}; +use tracing::{error, info}; use super::fetch::FetchTask; +use super::gc::GcTask; +use super::ingest_metastore::IngestMetastore; +use super::models::{Position, PrimaryShard, ReplicaShard, ShardStatus}; use super::replication::{ ReplicationClient, ReplicationClientTask, ReplicationTask, ReplicationTaskHandle, }; @@ -54,7 +64,10 @@ use crate::DocCommand; #[derive(Clone)] pub struct Ingester { self_node_id: NodeId, + metastore: Arc, ingester_pool: IngesterPool, + // TODO: There are a few places where we grab the two locks simultaneously. This will cause a + // deadlock. Let's fold `mrecordlog` into `IngesterState`. mrecordlog: Arc>, state: Arc>, replication_factor: usize, @@ -90,6 +103,7 @@ impl IngesterState { impl Ingester { pub async fn try_new( self_node_id: NodeId, + metastore: Arc, ingester_pool: Pool, wal_dir_path: &Path, replication_factor: usize, @@ -108,16 +122,128 @@ impl Ingester { replication_clients: HashMap::new(), replication_tasks: HashMap::new(), }; - let ingester = Self { + let mut ingester = Self { self_node_id, + metastore, ingester_pool, mrecordlog, state: Arc::new(RwLock::new(inner)), replication_factor, }; + info!( + replication_factor=%replication_factor, + wal_dir=%wal_dir_path.display(), + "spawning ingester" + ); + ingester.init().await?; + Ok(ingester) } + async fn init(&mut self) -> IngestV2Result<()> { + let mut mrecordlog_guard = self.mrecordlog.write().await; + let mut state = self.state.write().await; + + let queue_ids = mrecordlog_guard.list_queues(); + + let mut subrequests = Vec::new(); + + for queue_id in queue_ids { + let current_position = mrecordlog_guard + .current_position(queue_id) + .expect("queue should exist"); + + let (index_uid, source_id, shard_id) = + split_queue_id(queue_id).expect("queue ID should be well-formed"); + + let subrequest = CloseShardsSubrequest { + index_uid: index_uid.into(), + source_id, + shard_id, + shard_state: ShardState::Closed as i32, + replication_position_inclusive: current_position, + }; + subrequests.push(subrequest); + } + if subrequests.is_empty() { + return Ok(()); + } + let close_shards_request = CloseShardsRequest { subrequests }; + let close_shards_response = self.metastore.close_shards(close_shards_request).await?; + info!("closed {} shard(s)", close_shards_response.successes.len()); + + let mut gc_candidates: Vec = Vec::new(); + + for success in close_shards_response.successes { + let queue_id = success.queue_id(); + let publish_position_inclusive: Position = success.publish_position_inclusive.into(); + + if let Some(truncate_position) = publish_position_inclusive.offset() { + mrecordlog_guard + .truncate(&queue_id, truncate_position) + .await + .expect("queue should exist"); + } + let current_position: Position = mrecordlog_guard + .current_position(&queue_id) + .expect("queue should exist") + .into(); + let shard_status = ShardStatus { + shard_state: ShardState::Closed, + publish_position_inclusive, + replication_position_inclusive: current_position, + }; + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); + + if publish_position_inclusive >= current_position { + gc_candidates.push(queue_id.clone()) + } + if success.leader_id == self.self_node_id { + let primary_shard = PrimaryShard { + follower_id_opt: success.follower_id.map(|follower_id| follower_id.into()), + shard_state: ShardState::Closed, + publish_position_inclusive, + primary_position_inclusive: current_position, + replica_position_inclusive_opt: None, + shard_status_tx, + shard_status_rx, + }; + state.primary_shards.insert(queue_id, primary_shard); + } else { + let replica_shard = ReplicaShard { + leader_id: success.leader_id.into(), + shard_state: ShardState::Closed, + publish_position_inclusive, + replica_position_inclusive: current_position, + shard_status_tx, + shard_status_rx, + }; + state.replica_shards.insert(queue_id, replica_shard); + } + } + for failure in close_shards_response.failures { + if failure.failure_kind() == CloseShardsFailureKind::NotFound { + let queue_id = failure.queue_id(); + + if let Err(DeleteQueueError::IoError(error)) = + mrecordlog_guard.delete_queue(&queue_id).await + { + error!("failed to delete mrecordlog queue: {}", error); + } + } + } + if gc_candidates.is_empty() { + return Ok(()); + } + GcTask::spawn( + self.metastore.clone(), + self.mrecordlog.clone(), + self.state.clone(), + gc_candidates, + ); + Ok(()) + } + async fn init_primary_shard<'a>( &self, state: &'a mut IngesterState, @@ -174,11 +300,12 @@ impl Ingester { .try_send(open_message) .expect("The channel should be open and have capacity."); - let mut ingester = self.ingester_pool.get(follower_id).await.ok_or( - IngestV2Error::IngesterUnavailable { - ingester_id: follower_id.clone(), - }, - )?; + let mut ingester = + self.ingester_pool + .get(follower_id) + .ok_or(IngestV2Error::IngesterUnavailable { + ingester_id: follower_id.clone(), + })?; let mut ack_replication_stream = ingester .open_replication_stream(syn_replication_stream) .await?; @@ -212,7 +339,7 @@ impl IngesterService for Ingester { let mut state_guard = self.state.write().await; let mut persist_successes = Vec::with_capacity(persist_request.subrequests.len()); - // let mut persist_failures = Vec::new(); + let mut persist_failures = Vec::new(); let mut replicate_subrequests: HashMap> = HashMap::new(); let commit_type = persist_request.commit_type(); @@ -235,7 +362,14 @@ impl IngesterService for Ingester { .await? }; if primary_shard.shard_state.is_closed() { - // TODO + let persist_failure = PersistFailure { + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shard_id: subrequest.shard_id, + failure_kind: PersistFailureKind::ShardClosed as i32, + }; + persist_failures.push(persist_failure); + continue; } let from_position_inclusive = primary_shard.primary_position_inclusive; @@ -273,7 +407,7 @@ impl IngesterService for Ingester { state_guard .primary_shards .get_mut(&queue_id) - .expect("Primary shard should exist.") + .expect("primary shard should exist") .set_primary_position_inclusive(primary_position_inclusive); if let Some(follower_id) = follower_id { @@ -303,7 +437,7 @@ impl IngesterService for Ingester { let persist_response = PersistResponse { leader_id, successes: persist_successes, - failures: Vec::new(), // TODO + failures: persist_failures, }; return Ok(persist_response); } @@ -319,7 +453,7 @@ impl IngesterService for Ingester { let replication_client = state_guard .replication_clients .get(&follower_id) - .expect("The replication client should be initialized.") + .expect("the replication client should be initialized") .clone(); replicate_futures .push(async move { replication_client.replicate(replicate_request).await }); @@ -371,10 +505,10 @@ impl IngesterService for Ingester { .await .ok_or_else(|| IngestV2Error::Internal("syn replication stream aborted".to_string()))? .into_open_request() - .expect("The first message should be an open replication stream request."); + .expect("the first message should be an open replication stream request"); if open_replication_stream_request.follower_id != self.self_node_id { - return Err(IngestV2Error::Internal("Routing error".to_string())); + return Err(IngestV2Error::Internal("routing error".to_string())); } let leader_id: NodeId = open_replication_stream_request.leader_id.into(); let follower_id: NodeId = open_replication_stream_request.follower_id.into(); @@ -392,11 +526,12 @@ impl IngesterService for Ingester { ack_replication_stream_tx .send(Ok(ack_replication_message)) .await - .expect("Channel should be open and have enough capacity."); + .expect("channel should be open and have enough capacity"); let replication_task_handle = ReplicationTask::spawn( leader_id, follower_id, + self.metastore.clone(), self.mrecordlog.clone(), self.state.clone(), syn_replication_stream, @@ -417,7 +552,7 @@ impl IngesterService for Ingester { .read() .await .find_shard_status_rx(&queue_id) - .ok_or_else(|| IngestV2Error::Internal("shard not found.".to_string()))?; + .ok_or_else(|| IngestV2Error::Internal("shard not found".to_string()))?; let (service_stream, _fetch_task_handle) = FetchTask::spawn( open_fetch_stream_request, mrecordlog, @@ -437,7 +572,7 @@ impl IngesterService for Ingester { return Ok(ping_response); }; let follower_id: NodeId = follower_id.clone().into(); - let mut ingester = self.ingester_pool.get(&follower_id).await.ok_or({ + let mut ingester = self.ingester_pool.get(&follower_id).ok_or({ IngestV2Error::IngesterUnavailable { ingester_id: follower_id, } @@ -457,6 +592,7 @@ impl IngesterService for Ingester { truncate_request.leader_id, self.self_node_id ))); } + let mut gc_candidates: Vec = Vec::new(); let mut mrecordlog_guard = self.mrecordlog.write().await; let mut state_guard = self.state.write().await; @@ -470,9 +606,13 @@ impl IngesterService for Ingester { .truncate(&queue_id, subrequest.to_position_inclusive) .await .map_err(|error| { - IngestV2Error::Internal(format!("failed to truncate: {error:?}")) + IngestV2Error::Internal(format!("failed to truncate queue: {error:?}")) })?; primary_shard.set_publish_position_inclusive(subrequest.to_position_inclusive); + + if primary_shard.is_gc_candidate() { + gc_candidates.push(queue_id.clone()); + } } if let Some(replica_shard) = state_guard.replica_shards.get(&queue_id) { truncate_subrequests @@ -512,163 +652,6 @@ impl IngesterService for Ingester { } } -#[derive(Debug, Clone, Eq, PartialEq)] -pub(super) struct ShardStatus { - /// Current state of the shard. - pub shard_state: ShardState, - /// Position up to which indexers have indexed and published the records stored in the shard. - pub publish_position_inclusive: Position, - /// Position up to which the follower has acknowledged replication of the records written in - /// its log. - pub replication_position_inclusive: Position, -} - -impl Default for ShardStatus { - fn default() -> Self { - Self { - shard_state: ShardState::Open, - publish_position_inclusive: Position::default(), - replication_position_inclusive: Position::default(), - } - } -} - -#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub(super) enum Position { - #[default] - Beginning, - Offset(u64), -} - -impl Position { - pub fn offset(&self) -> Option { - match self { - Position::Beginning => None, - Position::Offset(offset) => Some(*offset), - } - } -} - -impl PartialEq for Position { - fn eq(&self, other: &u64) -> bool { - match self { - Position::Beginning => false, - Position::Offset(offset) => offset == other, - } - } -} - -impl PartialOrd for Position { - fn partial_cmp(&self, other: &u64) -> Option { - match self { - Position::Beginning => Some(cmp::Ordering::Less), - Position::Offset(offset) => offset.partial_cmp(other), - } - } -} - -impl From for Position { - fn from(offset: u64) -> Self { - Position::Offset(offset) - } -} - -impl From> for Position { - fn from(offset_opt: Option) -> Self { - match offset_opt { - Some(offset) => Position::Offset(offset), - None => Position::Beginning, - } - } -} - -/// Records the state of a primary shard managed by a leader. -pub(super) struct PrimaryShard { - /// Node ID of the ingester on which the replica shard is hosted. `None` if the replication - /// factor is 1. - pub follower_id_opt: Option, - /// Current state of the shard. - shard_state: ShardState, - /// Position up to which indexers have indexed and published the data stored in the shard. - /// It is updated asynchronously in a best effort manner by the indexers and indicates the - /// position up to which the log can be safely truncated. When the shard is closed, the - /// publish position has reached the replication position, and the deletion grace period has - /// passed, the shard can be safely deleted. - pub publish_position_inclusive: Position, - /// Position up to which the leader has written records in its log. - pub primary_position_inclusive: Position, - /// Position up to which the follower has acknowledged replication of the records written in - /// its log. - pub replica_position_inclusive_opt: Option, - /// Channel to notify readers that new records have been written to the shard. - pub shard_status_tx: watch::Sender, - pub shard_status_rx: watch::Receiver, -} - -impl PrimaryShard { - fn set_publish_position_inclusive(&mut self, publish_position_inclusive: impl Into) { - self.publish_position_inclusive = publish_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.publish_position_inclusive = self.publish_position_inclusive; - }); - } - - fn set_primary_position_inclusive(&mut self, primary_position_inclusive: impl Into) { - self.primary_position_inclusive = primary_position_inclusive.into(); - - // Notify readers if the replication factor is 1. - if self.follower_id_opt.is_none() { - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = self.primary_position_inclusive - }) - } - } - - fn set_replica_position_inclusive(&mut self, replica_position_inclusive: impl Into) { - assert!(self.follower_id_opt.is_some()); - - let replica_position_inclusive = replica_position_inclusive.into(); - self.replica_position_inclusive_opt = Some(replica_position_inclusive); - - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = replica_position_inclusive - }) - } -} - -/// Records the state of a replica shard managed by a follower. See [`PrimaryShard`] for more -/// details about the fields. -pub(super) struct ReplicaShard { - pub leader_id: NodeId, - pub(super) shard_state: ShardState, - pub(super) publish_position_inclusive: Position, - pub replica_position_inclusive: Position, - pub shard_status_tx: watch::Sender, - pub shard_status_rx: watch::Receiver, -} - -impl ReplicaShard { - pub fn set_publish_position_inclusive( - &mut self, - publish_position_inclusive: impl Into, - ) { - self.publish_position_inclusive = publish_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.publish_position_inclusive = self.publish_position_inclusive; - }); - } - - pub fn set_replica_position_inclusive( - &mut self, - replica_position_inclusive: impl Into, - ) { - self.replica_position_inclusive = replica_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = self.replica_position_inclusive - }); - } -} - // TODO pub(super) fn commit_doc() -> Bytes { let mut buffer = BytesMut::with_capacity(1); @@ -686,26 +669,207 @@ mod tests { IngesterServiceGrpcServer, IngesterServiceGrpcServerAdapter, PersistSubrequest, }; use quickwit_proto::ingest::DocBatchV2; + use quickwit_proto::metastore::{ + CloseShardsFailure, CloseShardsFailureKind, CloseShardsResponse, CloseShardsSuccess, + DeleteShardsResponse, + }; use quickwit_proto::types::queue_id; use tonic::transport::{Endpoint, Server}; use tower::timeout::Timeout; use super::*; + use crate::ingest_v2::ingest_metastore::MockIngestMetastore; use crate::ingest_v2::test_utils::{ MultiRecordLogTestExt, PrimaryShardTestExt, ReplicaShardTestExt, }; const NONE_REPLICA_POSITION: Option = None; + #[tokio::test] + async fn test_ingester_init() { + let tempdir = tempfile::tempdir().unwrap(); + let self_node_id: NodeId = "test-ingester-0".into(); + let mut mock_metastore = MockIngestMetastore::default(); + mock_metastore + .expect_close_shards() + .once() + .returning(|request| { + assert_eq!(request.subrequests.len(), 4); + let mut subrequests = request.subrequests; + subrequests.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); + + for (i, subrequest) in subrequests.iter().enumerate() { + assert_eq!(subrequest.index_uid, "test-index:0"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, i as u64 + 1); + assert_eq!(subrequest.shard_state(), ShardState::Closed); + } + assert!(subrequests[0].replication_position_inclusive.is_none()); + assert_eq!(subrequests[1].replication_position_inclusive, Some(1)); + assert_eq!(subrequests[2].replication_position_inclusive, Some(1)); + assert_eq!(subrequests[3].replication_position_inclusive, Some(1)); + + let response = CloseShardsResponse { + successes: vec![ + CloseShardsSuccess { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 2, + leader_id: "test-ingester-0".to_string(), + follower_id: Some("test-ingester-1".to_string()), + publish_position_inclusive: "1".to_string(), + }, + CloseShardsSuccess { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 3, + leader_id: "test-ingester-1".to_string(), + follower_id: Some("test-ingester-0".to_string()), + publish_position_inclusive: "0".to_string(), + }, + CloseShardsSuccess { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 4, + leader_id: "test-ingester-0".to_string(), + follower_id: Some("test-ingester-1".to_string()), + publish_position_inclusive: "".to_string(), + }, + ], + failures: vec![CloseShardsFailure { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + failure_kind: CloseShardsFailureKind::NotFound as i32, + failure_message: "shard not found".to_string(), + }], + }; + Ok(response) + }); + mock_metastore + .expect_delete_shards() + .once() + .returning(|request| { + assert_eq!(request.subrequests.len(), 1); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.index_uid, "test-index:0"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_ids, [2]); + + let response = DeleteShardsResponse {}; + Ok(response) + }); + let metastore = Arc::new(mock_metastore); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 2; + let mut ingester = Ingester::try_new( + self_node_id.clone(), + metastore, + ingester_pool, + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let mut mrecordlog_guard = ingester.mrecordlog.write().await; + + let queue_ids: Vec = (1..=4) + .map(|shard_id| queue_id("test-index:0", "test-source", shard_id)) + .collect(); + + for queue_id in &queue_ids { + mrecordlog_guard.create_queue(queue_id).await.unwrap(); + } + let records = [ + Bytes::from_static(b"test-doc-200"), + Bytes::from_static(b"test-doc-201"), + ]; + mrecordlog_guard + .append_records(&queue_ids[1], None, records.into_iter()) + .await + .unwrap(); + let records = [ + Bytes::from_static(b"test-doc-300"), + Bytes::from_static(b"test-doc-301"), + ]; + mrecordlog_guard + .append_records(&queue_ids[2], None, records.into_iter()) + .await + .unwrap(); + let records = [ + Bytes::from_static(b"test-doc-400"), + Bytes::from_static(b"test-doc-401"), + ]; + mrecordlog_guard + .append_records(&queue_ids[3], None, records.into_iter()) + .await + .unwrap(); + drop(mrecordlog_guard); + + ingester.init().await.unwrap(); + + let mrecordlog_guard = ingester.mrecordlog.read().await; + assert!(!mrecordlog_guard.queue_exists(&queue_ids[0])); + drop(mrecordlog_guard); + + let state_guard = ingester.state.read().await; + assert_eq!(state_guard.primary_shards.len(), 2); + assert_eq!(state_guard.replica_shards.len(), 1); + + let primary_shard_2 = state_guard.primary_shards.get(&queue_ids[1]).unwrap(); + assert_eq!( + primary_shard_2.publish_position_inclusive, + Position::Offset(1) + ); + assert!(primary_shard_2.shard_state.is_closed()); + primary_shard_2.assert_positions(Some(1), NONE_REPLICA_POSITION); + + let primary_shard_4 = state_guard.primary_shards.get(&queue_ids[3]).unwrap(); + assert_eq!( + primary_shard_4.publish_position_inclusive, + Position::Beginning + ); + assert!(primary_shard_4.shard_state.is_closed()); + primary_shard_2.assert_positions(Some(1), NONE_REPLICA_POSITION); + + let replica_shard_3 = state_guard.replica_shards.get(&queue_ids[2]).unwrap(); + assert_eq!( + replica_shard_3.publish_position_inclusive, + Position::Offset(0), + ); + assert_eq!( + replica_shard_3.replica_position_inclusive, + Position::Offset(1), + ); + assert!(replica_shard_3.shard_state.is_closed()); + + drop(state_guard); + + // Wait for the GC task to run. + tokio::time::sleep(Duration::from_millis(100)).await; + + let state_guard = ingester.state.read().await; + assert_eq!(state_guard.primary_shards.len(), 1); + assert!(!state_guard.primary_shards.contains_key(&queue_ids[1])); + + let mrecordlog_guard = ingester.mrecordlog.read().await; + assert!(!mrecordlog_guard.queue_exists(&queue_ids[1])); + } + #[tokio::test] async fn test_ingester_persist() { let tempdir = tempfile::tempdir().unwrap(); - let node_id: NodeId = "test-ingester-0".into(); + let self_node_id: NodeId = "test-ingester-0".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( - node_id.clone(), + self_node_id.clone(), + metastore, ingester_pool, wal_dir_path, replication_factor, @@ -714,7 +878,7 @@ mod tests { .unwrap(); let persist_request = PersistRequest { - leader_id: node_id.to_string(), + leader_id: self_node_id.to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ PersistSubrequest { @@ -781,12 +945,14 @@ mod tests { #[tokio::test] async fn test_ingester_open_replication_stream() { let tempdir = tempfile::tempdir().unwrap(); - let node_id: NodeId = "test-follower".into(); + let self_node_id: NodeId = "test-follower".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( - node_id.clone(), + self_node_id.clone(), + metastore, ingester_pool, wal_dir_path, replication_factor, @@ -823,11 +989,13 @@ mod tests { async fn test_ingester_persist_replicate() { let tempdir = tempfile::tempdir().unwrap(); let leader_id: NodeId = "test-leader".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 2; let mut leader = Ingester::try_new( leader_id.clone(), + metastore.clone(), ingester_pool.clone(), wal_dir_path, replication_factor, @@ -841,6 +1009,7 @@ mod tests { let replication_factor = 2; let follower = Ingester::try_new( follower_id.clone(), + metastore, ingester_pool.clone(), wal_dir_path, replication_factor, @@ -848,12 +1017,10 @@ mod tests { .await .unwrap(); - ingester_pool - .insert( - follower_id.clone(), - IngesterServiceClient::new(follower.clone()), - ) - .await; + ingester_pool.insert( + follower_id.clone(), + IngesterServiceClient::new(follower.clone()), + ); let persist_request = PersistRequest { leader_id: "test-leader".to_string(), @@ -927,11 +1094,13 @@ mod tests { async fn test_ingester_persist_replicate_grpc() { let tempdir = tempfile::tempdir().unwrap(); let leader_id: NodeId = "test-leader".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 2; let mut leader = Ingester::try_new( leader_id.clone(), + metastore.clone(), ingester_pool.clone(), wal_dir_path, replication_factor, @@ -959,6 +1128,7 @@ mod tests { let replication_factor = 2; let follower = Ingester::try_new( follower_id.clone(), + metastore, ingester_pool.clone(), wal_dir_path, replication_factor, @@ -985,9 +1155,7 @@ mod tests { ); let follower_grpc_client = IngesterServiceClient::from_channel(follower_channel); - ingester_pool - .insert(follower_id.clone(), follower_grpc_client) - .await; + ingester_pool.insert(follower_id.clone(), follower_grpc_client); let persist_request = PersistRequest { leader_id: "test-leader".to_string(), @@ -1092,12 +1260,14 @@ mod tests { #[tokio::test] async fn test_ingester_open_fetch_stream() { let tempdir = tempfile::tempdir().unwrap(); - let node_id: NodeId = "test-ingester-0".into(); + let self_node_id: NodeId = "test-ingester-0".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( - node_id.clone(), + self_node_id.clone(), + metastore, ingester_pool, wal_dir_path, replication_factor, @@ -1106,7 +1276,7 @@ mod tests { .unwrap(); let persist_request = PersistRequest { - leader_id: node_id.to_string(), + leader_id: self_node_id.to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ PersistSubrequest { @@ -1153,7 +1323,7 @@ mod tests { assert_eq!(fetch_response.from_position_inclusive, 0); let persist_request = PersistRequest { - leader_id: node_id.to_string(), + leader_id: self_node_id.to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![PersistSubrequest { index_uid: "test-index:0".to_string(), @@ -1181,12 +1351,14 @@ mod tests { #[tokio::test] async fn test_ingester_truncate() { let tempdir = tempfile::tempdir().unwrap(); - let node_id: NodeId = "test-ingester-0".into(); + let self_node_id: NodeId = "test-ingester-0".into(); + let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( - node_id.clone(), + self_node_id.clone(), + metastore, ingester_pool, wal_dir_path, replication_factor, @@ -1199,11 +1371,11 @@ mod tests { let mut ingester_state = ingester.state.write().await; ingester - .init_primary_shard(&mut ingester_state, &queue_id_00, &node_id, None) + .init_primary_shard(&mut ingester_state, &queue_id_00, &self_node_id, None) .await .unwrap(); ingester - .init_primary_shard(&mut ingester_state, &queue_id_01, &node_id, None) + .init_primary_shard(&mut ingester_state, &queue_id_01, &self_node_id, None) .await .unwrap(); @@ -1234,7 +1406,7 @@ mod tests { drop(mrecordlog_guard); let truncate_request = TruncateRequest { - leader_id: node_id.to_string(), + leader_id: self_node_id.to_string(), subrequests: vec![ TruncateSubrequest { index_uid: "test-index:0".to_string(), diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index a5b57bc79e1..d8e593e257f 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -18,7 +18,10 @@ // along with this program. If not, see . mod fetch; +mod gc; +mod ingest_metastore; mod ingester; +mod models; mod replication; mod router; mod shard_table; @@ -32,6 +35,7 @@ use quickwit_proto::ingest::DocBatchV2; use quickwit_proto::types::NodeId; pub use self::fetch::MultiFetchStream; +pub use self::ingest_metastore::IngestMetastore; pub use self::ingester::Ingester; pub use self::router::IngestRouter; @@ -66,14 +70,11 @@ impl DocBatchBuilderV2 { } } + /// Returns the capacity of the underlying buffer, expressed in bytes. pub fn capacity(&self) -> usize { self.doc_buffer.capacity() } - pub fn is_empty(&self) -> bool { - self.doc_lengths.is_empty() - } - fn num_bytes(&self) -> usize { self.doc_buffer.len() } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/models.rs b/quickwit/quickwit-ingest/src/ingest_v2/models.rs new file mode 100644 index 00000000000..0ef85c19958 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/models.rs @@ -0,0 +1,278 @@ +// 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::cmp; + +use quickwit_proto::ingest::ShardState; +use quickwit_proto::NodeId; +use tokio::sync::watch; + +#[derive(Debug, Clone, Eq, PartialEq)] +pub(super) struct ShardStatus { + /// Current state of the shard. + pub shard_state: ShardState, + /// Position up to which indexers have indexed and published the records stored in the shard. + pub publish_position_inclusive: Position, + /// Position up to which the follower has acknowledged replication of the records written in + /// its log. + pub replication_position_inclusive: Position, +} + +impl Default for ShardStatus { + fn default() -> Self { + Self { + shard_state: ShardState::Open, + publish_position_inclusive: Position::default(), + replication_position_inclusive: Position::default(), + } + } +} + +#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd, Hash)] +pub(super) enum Position { + #[default] + Beginning, + Offset(u64), +} + +impl Position { + pub fn offset(&self) -> Option { + match self { + Position::Beginning => None, + Position::Offset(offset) => Some(*offset), + } + } +} + +impl PartialEq for Position { + fn eq(&self, other: &u64) -> bool { + match self { + Position::Beginning => false, + Position::Offset(offset) => offset == other, + } + } +} + +impl PartialOrd for Position { + fn partial_cmp(&self, other: &u64) -> Option { + match self { + Position::Beginning => Some(cmp::Ordering::Less), + Position::Offset(offset) => offset.partial_cmp(other), + } + } +} + +impl From for Position { + fn from(offset: u64) -> Self { + Position::Offset(offset) + } +} + +impl From> for Position { + fn from(offset_opt: Option) -> Self { + match offset_opt { + Some(offset) => Position::Offset(offset), + None => Position::Beginning, + } + } +} + +impl From for Position { + fn from(position_str: String) -> Self { + if position_str.is_empty() { + Position::Beginning + } else { + let offset = position_str + .parse::() + .expect("position should be a u64"); + Position::Offset(offset) + } + } +} + +/// Records the state of a primary shard managed by a leader. +pub(super) struct PrimaryShard { + /// Node ID of the ingester on which the replica shard is hosted. `None` if the replication + /// factor is 1. + pub follower_id_opt: Option, + /// Current state of the shard. + pub shard_state: ShardState, + /// Position up to which indexers have indexed and published the data stored in the shard. + /// It is updated asynchronously in a best effort manner by the indexers and indicates the + /// position up to which the log can be safely truncated. When the shard is closed, the + /// publish position has reached the replication position, and the deletion grace period has + /// passed, the shard can be safely deleted (see [`GcTask`] more details about the deletion + /// logic). + pub publish_position_inclusive: Position, + /// Position up to which the leader has written records in its log. + pub primary_position_inclusive: Position, + /// Position up to which the follower has acknowledged replication of the records written in + /// its log. + pub replica_position_inclusive_opt: Option, + /// Channel to notify readers that new records have been written to the shard. + pub shard_status_tx: watch::Sender, + pub shard_status_rx: watch::Receiver, +} + +impl PrimaryShard { + pub fn is_gc_candidate(&self) -> bool { + self.shard_state.is_closed() + && self.publish_position_inclusive >= self.primary_position_inclusive + } + + pub fn set_publish_position_inclusive( + &mut self, + publish_position_inclusive: impl Into, + ) { + self.publish_position_inclusive = publish_position_inclusive.into(); + self.shard_status_tx.send_modify(|shard_status| { + shard_status.publish_position_inclusive = self.publish_position_inclusive; + }); + } + + pub fn set_primary_position_inclusive( + &mut self, + primary_position_inclusive: impl Into, + ) { + self.primary_position_inclusive = primary_position_inclusive.into(); + + // Notify readers if the replication factor is 1. + if self.follower_id_opt.is_none() { + self.shard_status_tx.send_modify(|shard_status| { + shard_status.replication_position_inclusive = self.primary_position_inclusive + }) + } + } + + pub fn set_replica_position_inclusive( + &mut self, + replica_position_inclusive: impl Into, + ) { + assert!(self.follower_id_opt.is_some()); + + let replica_position_inclusive = replica_position_inclusive.into(); + self.replica_position_inclusive_opt = Some(replica_position_inclusive); + + self.shard_status_tx.send_modify(|shard_status| { + shard_status.replication_position_inclusive = replica_position_inclusive + }) + } +} + +/// Records the state of a replica shard managed by a follower. See [`PrimaryShard`] for more +/// details about the fields. +pub(super) struct ReplicaShard { + pub leader_id: NodeId, + pub shard_state: ShardState, + pub publish_position_inclusive: Position, + pub replica_position_inclusive: Position, + pub shard_status_tx: watch::Sender, + pub shard_status_rx: watch::Receiver, +} + +impl ReplicaShard { + pub fn is_gc_candidate(&self) -> bool { + self.shard_state.is_closed() + && self.publish_position_inclusive >= self.replica_position_inclusive + } + + pub fn set_publish_position_inclusive( + &mut self, + publish_position_inclusive: impl Into, + ) { + self.publish_position_inclusive = publish_position_inclusive.into(); + self.shard_status_tx.send_modify(|shard_status| { + shard_status.publish_position_inclusive = self.publish_position_inclusive; + }); + } + + pub fn set_replica_position_inclusive( + &mut self, + replica_position_inclusive: impl Into, + ) { + self.replica_position_inclusive = replica_position_inclusive.into(); + self.shard_status_tx.send_modify(|shard_status| { + shard_status.replication_position_inclusive = self.replica_position_inclusive + }); + } +} + +#[cfg(test)] +impl PrimaryShard { + pub(crate) fn for_test( + follower_id_opt: Option<&str>, + shard_state: ShardState, + publish_position_inclusive: impl Into, + primary_position_inclusive: impl Into, + replica_position_inclusive_opt: Option>, + ) -> Self { + let publish_position_inclusive: Position = publish_position_inclusive.into(); + let primary_position_inclusive: Position = primary_position_inclusive.into(); + let replica_position_inclusive_opt: Option = + replica_position_inclusive_opt.map(Into::into); + let replication_position_inclusive = + replica_position_inclusive_opt.unwrap_or(primary_position_inclusive); + + let shard_status = ShardStatus { + shard_state, + publish_position_inclusive, + replication_position_inclusive, + }; + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); + + Self { + follower_id_opt: follower_id_opt.map(Into::into), + shard_state, + publish_position_inclusive, + primary_position_inclusive, + replica_position_inclusive_opt, + shard_status_tx, + shard_status_rx, + } + } +} + +#[cfg(test)] +impl ReplicaShard { + pub(crate) fn for_test( + leader_id: &str, + shard_state: ShardState, + publish_position_inclusive: impl Into, + replica_position_inclusive: impl Into, + ) -> Self { + let publish_position_inclusive: Position = publish_position_inclusive.into(); + let replica_position_inclusive: Position = replica_position_inclusive.into(); + + let shard_status = ShardStatus { + shard_state, + publish_position_inclusive, + replication_position_inclusive: replica_position_inclusive, + }; + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); + + Self { + leader_id: leader_id.into(), + shard_state, + publish_position_inclusive, + replica_position_inclusive, + shard_status_tx, + shard_status_rx, + } + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 5f4bc9c5143..f292829b310 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -29,10 +29,14 @@ use quickwit_proto::ingest::ingester::{ }; use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; use quickwit_proto::types::NodeId; +use quickwit_proto::QueueId; use tokio::sync::{mpsc, oneshot, watch, RwLock}; use tokio::task::JoinHandle; -use super::ingester::{commit_doc, IngesterState, Position, ReplicaShard, ShardStatus}; +use super::gc::GcTask; +use super::ingest_metastore::IngestMetastore; +use super::ingester::{commit_doc, IngesterState}; +use super::models::{Position, ReplicaShard, ShardStatus}; use crate::metrics::INGEST_METRICS; /// A replication request is sent by the leader to its follower to update the state of a replica @@ -191,6 +195,7 @@ pub(super) struct ReplicationTaskHandle { pub(super) struct ReplicationTask { leader_id: NodeId, follower_id: NodeId, + metastore: Arc, mrecordlog: Arc>, state: Arc>, syn_replication_stream: ServiceStream, @@ -201,6 +206,7 @@ impl ReplicationTask { pub fn spawn( leader_id: NodeId, follower_id: NodeId, + metastore: Arc, mrecordlog: Arc>, state: Arc>, syn_replication_stream: ServiceStream, @@ -209,6 +215,7 @@ impl ReplicationTask { let mut replication_task = Self { leader_id, follower_id, + metastore, mrecordlog, state, syn_replication_stream, @@ -356,6 +363,7 @@ impl ReplicationTask { &mut self, truncate_request: TruncateRequest, ) -> IngestV2Result { + let mut gc_candidates: Vec = Vec::new(); let mut mrecordlog_guard = self.mrecordlog.write().await; let mut state_guard = self.state.write().await; @@ -371,8 +379,18 @@ impl ReplicationTask { })?; replica_shard .set_publish_position_inclusive(truncate_subrequest.to_position_inclusive); + + if replica_shard.is_gc_candidate() { + gc_candidates.push(queue_id); + } } } + GcTask::spawn( + self.metastore.clone(), + self.mrecordlog.clone(), + self.state.clone(), + gc_candidates, + ); let truncate_response = TruncateResponse {}; Ok(truncate_response) } @@ -426,6 +444,7 @@ mod tests { use quickwit_proto::types::queue_id; use super::*; + use crate::ingest_v2::ingest_metastore::MockIngestMetastore; use crate::ingest_v2::test_utils::{MultiRecordLogTestExt, ReplicaShardTestExt}; #[tokio::test] @@ -530,6 +549,7 @@ mod tests { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); let tempdir = tempfile::tempdir().unwrap(); + let metastore = Arc::new(MockIngestMetastore::default()); let mrecordlog = Arc::new(RwLock::new( MultiRecordLog::open(tempdir.path()).await.unwrap(), )); @@ -544,6 +564,7 @@ mod tests { let _replication_task_handle = ReplicationTask::spawn( leader_id, follower_id, + metastore, mrecordlog.clone(), state.clone(), syn_replication_stream, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 8204cfd0858..a54025218c6 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -17,7 +17,6 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::borrow::Borrow; use std::collections::HashMap; use std::fmt; use std::sync::Arc; @@ -26,17 +25,20 @@ use async_trait::async_trait; use futures::stream::FuturesUnordered; use futures::StreamExt; use quickwit_proto::control_plane::{ - ControlPlaneService, ControlPlaneServiceClient, GetOpenShardsRequest, GetOpenShardsSubrequest, + ControlPlaneService, ControlPlaneServiceClient, GetOrCreateOpenShardsRequest, + GetOrCreateOpenShardsSubrequest, }; use quickwit_proto::ingest::ingester::{IngesterService, PersistRequest, PersistSubrequest}; -use quickwit_proto::ingest::router::{IngestRequestV2, IngestResponseV2, IngestRouterService}; +use quickwit_proto::ingest::router::{ + IngestRequestV2, IngestResponseV2, IngestRouterService, IngestSubrequest, +}; use quickwit_proto::ingest::IngestV2Result; use quickwit_proto::types::NodeId; use quickwit_proto::IndexUid; use tokio::sync::RwLock; use super::shard_table::ShardTable; -use super::{DocBatchBuilderV2, IngesterPool}; +use super::IngesterPool; type LeaderId = String; @@ -81,43 +83,57 @@ impl IngestRouter { } } - async fn refresh_shard_table( - &mut self, - ingest_request: &IngestRequestV2, - ) -> IngestV2Result<()> { + /// Inspects the shard table for each subrequest and returns the appropriate + /// [`GetOrCreateOpenShardsRequest`] request if open shards do not exist for all the them. + async fn make_get_or_create_open_shard_request( + &self, + subrequests: &[IngestSubrequest], + ) -> Option { let state_guard = self.state.read().await; - - let shard_table = &state_guard.shard_table; let mut get_open_shards_subrequests = Vec::new(); - for ingest_subrequest in &ingest_request.subrequests { - if !shard_table - .contains_entry(&*ingest_subrequest.index_id, &ingest_subrequest.source_id) + for subrequest in subrequests { + if !state_guard + .shard_table + .contains_entry(&subrequest.index_id, &subrequest.source_id) { - let subrequest = GetOpenShardsSubrequest { - index_id: ingest_subrequest.index_id.clone(), - source_id: ingest_subrequest.source_id.clone(), + let subrequest = GetOrCreateOpenShardsSubrequest { + index_id: subrequest.index_id.clone(), + source_id: subrequest.source_id.clone(), + closed_shards: Vec::new(), // TODO }; get_open_shards_subrequests.push(subrequest); } } if get_open_shards_subrequests.is_empty() { - return Ok(()); + return None; } drop(state_guard); - let request = GetOpenShardsRequest { + let request = GetOrCreateOpenShardsRequest { subrequests: get_open_shards_subrequests, unavailable_ingesters: Vec::new(), }; - let response = self.control_plane.get_open_shards(request).await?; + Some(request) + } + + /// Issues a [`GetOrCreateOpenShardsRequest`] request to the control plane and populates the + /// shard table according to the response received. + async fn populate_shard_table( + &mut self, + request: GetOrCreateOpenShardsRequest, + ) -> IngestV2Result<()> { + let response = self + .control_plane + .get_or_create_open_shards(request) + .await?; let mut state_guard = self.state.write().await; for subresponse in response.subresponses { let index_uid: IndexUid = subresponse.index_uid.into(); let index_id = index_uid.index_id().to_string(); - state_guard.shard_table.update_entry( + state_guard.shard_table.insert_shards( index_id, subresponse.source_id, subresponse.open_shards, @@ -133,10 +149,15 @@ impl IngestRouterService for IngestRouter { &mut self, ingest_request: IngestRequestV2, ) -> IngestV2Result { - self.refresh_shard_table(&ingest_request).await?; - - let mut doc_batch_builders: Vec = Vec::new(); - let mut persist_subrequests: HashMap<&LeaderId, Vec> = HashMap::new(); + if let Some(get_or_create_open_shards_request) = self + .make_get_or_create_open_shard_request(&ingest_request.subrequests) + .await + { + self.populate_shard_table(get_or_create_open_shards_request) + .await?; + } + let mut per_leader_persist_subrequests: HashMap<&LeaderId, Vec> = + HashMap::new(); let state_guard = self.state.read().await; @@ -144,58 +165,29 @@ impl IngestRouterService for IngestRouter { // lines, validate, transform and then pack the docs into compressed batches routed // to the right shards. for ingest_subrequest in ingest_request.subrequests { - let table_entry = state_guard + let shard = state_guard .shard_table .find_entry(&*ingest_subrequest.index_id, &ingest_subrequest.source_id) - .expect("TODO"); - - if table_entry.len() == 1 { - let shard = &table_entry.shards()[0]; - let persist_subrequest = PersistSubrequest { - index_uid: shard.index_uid.clone(), - source_id: ingest_subrequest.source_id, - shard_id: shard.shard_id, - follower_id: shard.follower_id.clone(), - doc_batch: ingest_subrequest.doc_batch, - }; - persist_subrequests - .entry(&shard.leader_id) - .or_default() - .push(persist_subrequest); - continue; - } - doc_batch_builders.resize_with(table_entry.len(), DocBatchBuilderV2::default); - - for (i, doc) in ingest_subrequest.docs().enumerate() { - let shard_idx = i % table_entry.len(); - doc_batch_builders[shard_idx].add_doc(doc.borrow()); - } - for (shard, doc_batch_builder) in table_entry - .shards() - .iter() - .zip(doc_batch_builders.drain(..)) - { - if !doc_batch_builder.is_empty() { - let doc_batch = doc_batch_builder.build(); - let persist_subrequest = PersistSubrequest { - index_uid: shard.index_uid.clone(), - source_id: ingest_subrequest.source_id.clone(), - shard_id: shard.shard_id, - follower_id: shard.follower_id.clone(), - doc_batch: Some(doc_batch), - }; - persist_subrequests - .entry(&shard.leader_id) - .or_default() - .push(persist_subrequest); - } - } + .expect("TODO") + .next_shard_round_robin(); + + let persist_subrequest = PersistSubrequest { + index_uid: shard.index_uid.clone(), + source_id: ingest_subrequest.source_id, + shard_id: shard.shard_id, + follower_id: shard.follower_id.clone(), + doc_batch: ingest_subrequest.doc_batch, + }; + per_leader_persist_subrequests + .entry(&shard.leader_id) + .or_default() + .push(persist_subrequest); } let mut persist_futures = FuturesUnordered::new(); - for (leader_id, subrequests) in persist_subrequests { + 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).await.expect("TODO"); + let mut ingester = self.ingester_pool.get(&leader_id).expect("TODO"); let persist_request = PersistRequest { leader_id: leader_id.into(), @@ -221,7 +213,7 @@ impl IngestRouterService for IngestRouter { #[cfg(test)] mod tests { use bytes::Bytes; - use quickwit_proto::control_plane::{GetOpenShardsResponse, GetOpenShardsSubresponse}; + use quickwit_proto::control_plane::{GetOpenShardsSubresponse, GetOrCreateOpenShardsResponse}; use quickwit_proto::ingest::ingester::{IngesterServiceClient, PersistResponse}; use quickwit_proto::ingest::router::IngestSubrequest; use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard}; @@ -229,15 +221,85 @@ mod tests { use super::*; #[tokio::test] - async fn test_router_refresh_shard_table() { + async fn test_router_make_get_or_create_open_shard_request() { + let self_node_id = "test-router".into(); + let control_plane: ControlPlaneServiceClient = ControlPlaneServiceClient::mock().into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + let router = IngestRouter::new( + self_node_id, + control_plane, + ingester_pool.clone(), + replication_factor, + ); + let get_or_create_open_shard_request_opt = + router.make_get_or_create_open_shard_request(&[]).await; + assert!(get_or_create_open_shard_request_opt.is_none()); + + let ingest_subrequests = [ + IngestSubrequest { + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + IngestSubrequest { + 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) + .await + .unwrap(); + + assert_eq!(get_or_create_open_shard_request.subrequests.len(), 2); + + let subrequest = &get_or_create_open_shard_request.subrequests[0]; + assert_eq!(subrequest.index_id, "test-index-0"); + assert_eq!(subrequest.source_id, "test-source"); + + let subrequest = &get_or_create_open_shard_request.subrequests[1]; + assert_eq!(subrequest.index_id, "test-index-1"); + assert_eq!(subrequest.source_id, "test-source"); + + let mut state_guard = router.state.write().await; + + state_guard.shard_table.insert_shards( + "test-index-0", + "test-source", + vec![Shard { + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + ..Default::default() + }], + ); + drop(state_guard); + + let get_or_create_open_shard_request = router + .make_get_or_create_open_shard_request(&ingest_subrequests) + .await + .unwrap(); + + assert_eq!(get_or_create_open_shard_request.subrequests.len(), 1); + + let subrequest = &get_or_create_open_shard_request.subrequests[1]; + assert_eq!(subrequest.index_id, "test-index-1"); + assert_eq!(subrequest.source_id, "test-source"); + } + + #[tokio::test] + async fn test_router_populate_shard_table() { let self_node_id = "test-router".into(); let mut control_plane_mock = ControlPlaneServiceClient::mock(); control_plane_mock - .expect_get_open_shards() + .expect_get_or_create_open_shards() .once() .returning(|request| { assert_eq!(request.subrequests.len(), 2); + let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_id, "test-index-0"); assert_eq!(subrequest.source_id, "test-source"); @@ -246,7 +308,7 @@ mod tests { assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); - let response = GetOpenShardsResponse { + let response = GetOrCreateOpenShardsResponse { subresponses: vec![GetOpenShardsSubresponse { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), @@ -267,30 +329,35 @@ mod tests { ingester_pool.clone(), replication_factor, ); - - let ingest_request = IngestRequestV2 { + let get_or_create_open_shards_request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), - commit_type: CommitTypeV2::Auto as i32, + unavailable_ingesters: Vec::new(), }; - router.refresh_shard_table(&ingest_request).await.unwrap(); + router + .populate_shard_table(get_or_create_open_shards_request) + .await + .unwrap(); assert!(router.state.read().await.shard_table.is_empty()); - let ingest_request = IngestRequestV2 { + let get_or_create_open_shards_request = GetOrCreateOpenShardsRequest { subrequests: vec![ - IngestSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), ..Default::default() }, - IngestSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), ..Default::default() }, ], - commit_type: CommitTypeV2::Auto as i32, + unavailable_ingesters: Vec::new(), }; - router.refresh_shard_table(&ingest_request).await.unwrap(); + router + .populate_shard_table(get_or_create_open_shards_request) + .await + .unwrap(); let state_guard = router.state.read().await; let shard_table = &state_guard.shard_table; @@ -305,7 +372,7 @@ mod tests { let mut control_plane_mock = ControlPlaneServiceClient::mock(); control_plane_mock - .expect_get_open_shards() + .expect_get_or_create_open_shards() .once() .returning(|request| { assert_eq!(request.subrequests.len(), 1); @@ -313,7 +380,7 @@ mod tests { assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); - let response = GetOpenShardsResponse { + let response = GetOrCreateOpenShardsResponse { subresponses: vec![GetOpenShardsSubresponse { index_uid: "test-index-1:1".to_string(), source_id: "test-source".to_string(), @@ -334,22 +401,25 @@ mod tests { let control_plane: ControlPlaneServiceClient = control_plane_mock.into(); router.control_plane = control_plane; - let ingest_request = IngestRequestV2 { + let get_or_create_open_shards_request = GetOrCreateOpenShardsRequest { subrequests: vec![ - IngestSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), ..Default::default() }, - IngestSubrequest { + GetOrCreateOpenShardsSubrequest { index_id: "test-index-1".to_string(), source_id: "test-source".to_string(), ..Default::default() }, ], - commit_type: CommitTypeV2::Auto as i32, + unavailable_ingesters: Vec::new(), }; - router.refresh_shard_table(&ingest_request).await.unwrap(); + router + .populate_shard_table(get_or_create_open_shards_request) + .await + .unwrap(); let state_guard = router.state.read().await; let shard_table = &state_guard.shard_table; @@ -383,7 +453,7 @@ mod tests { ); let mut state_guard = router.state.write().await; - state_guard.shard_table.update_entry( + state_guard.shard_table.insert_shards( "test-index-0", "test-source", vec![Shard { @@ -393,7 +463,7 @@ mod tests { ..Default::default() }], ); - state_guard.shard_table.update_entry( + state_guard.shard_table.insert_shards( "test-index-1", "test-source", vec![ @@ -422,7 +492,7 @@ mod tests { .returning(|mut request| { assert_eq!(request.leader_id, "test-ingester-0"); assert_eq!(request.subrequests.len(), 2); - assert_eq!(request.commit_type, CommitTypeV2::Auto as i32); + assert_eq!(request.commit_type(), CommitTypeV2::Auto); request .subrequests @@ -458,9 +528,7 @@ mod tests { Ok(response) }); let ingester_0: IngesterServiceClient = ingester_mock_0.into(); - ingester_pool - .insert("test-ingester-0".into(), ingester_0.clone()) - .await; + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let mut ingester_mock_1 = IngesterServiceClient::mock(); ingester_mock_1 @@ -469,7 +537,7 @@ mod tests { .returning(|request| { assert_eq!(request.leader_id, "test-ingester-1"); assert_eq!(request.subrequests.len(), 1); - assert_eq!(request.commit_type, CommitTypeV2::Auto as i32); + assert_eq!(request.commit_type(), CommitTypeV2::Auto); let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index-1:1"); @@ -490,9 +558,7 @@ mod tests { Ok(response) }); let ingester_1: IngesterServiceClient = ingester_mock_1.into(); - ingester_pool - .insert("test-ingester-1".into(), ingester_1) - .await; + ingester_pool.insert("test-ingester-1".into(), ingester_1); let ingest_request = IngestRequestV2 { commit_type: CommitTypeV2::Auto as i32, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs index aeb90c7d77d..1ecd16f0e19 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs @@ -18,6 +18,7 @@ // along with this program. If not, see . use std::collections::HashMap; +use std::sync::atomic::{AtomicUsize, Ordering}; use quickwit_proto::ingest::Shard; use quickwit_proto::types::SourceId; @@ -25,26 +26,42 @@ use quickwit_proto::IndexId; /// A set of open shards for a given index and source. #[derive(Debug, Default)] -pub(crate) struct ShardTableEntry { +pub(super) struct ShardTableEntry { shards: Vec, + round_robin_idx: AtomicUsize, } impl ShardTableEntry { /// Creates a new entry and ensures that the shards are open and unique. + /// + /// # Panics + /// + /// Panics if `shards` is empty after filtering out closed shards and deduplicating by shard ID. pub fn new(mut shards: Vec) -> Self { shards.retain(|shard| shard.is_open()); shards.sort_unstable_by_key(|shard| shard.shard_id); shards.dedup_by_key(|shard| shard.shard_id); - Self { shards } + assert!(!shards.is_empty(), "`shards` should not be empty"); + + Self { + shards, + round_robin_idx: AtomicUsize::default(), + } + } + + /// Returns the next shard in round-robin order ensuring that the leader is available. + pub fn next_shard_round_robin(&self) -> &Shard { + let shard_idx = self.round_robin_idx.fetch_add(1, Ordering::Relaxed); + &self.shards[shard_idx % self.shards.len()] } - /// Returns the number of shards that make up the entry. + #[cfg(test)] pub fn len(&self) -> usize { self.shards.len() } - /// Returns the shards that make up the entry. + #[cfg(test)] pub fn shards(&self) -> &[Shard] { &self.shards } @@ -52,7 +69,7 @@ impl ShardTableEntry { /// A table of shard entries indexed by index UID and source ID. #[derive(Debug, Default)] -pub(crate) struct ShardTable { +pub(super) struct ShardTable { table: HashMap<(IndexId, SourceId), ShardTableEntry>, } @@ -75,7 +92,7 @@ impl ShardTable { self.table.get(&key) } - pub fn update_entry( + pub fn insert_shards( &mut self, index_id: impl Into, source_id: impl Into, @@ -107,28 +124,32 @@ mod tests { let mut table = ShardTable::default(); assert!(!table.contains_entry("test-index", "test-source")); - table.update_entry( + table.insert_shards( "test-index", "test-source", vec![ Shard { index_uid: "test-index:0".to_string(), shard_id: 0, + leader_id: "node-0".to_string(), ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), shard_id: 1, + leader_id: "node-1".to_string(), ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), shard_id: 0, + leader_id: "node-0".to_string(), ..Default::default() }, Shard { index_uid: "test-index:0".to_string(), shard_id: 2, + leader_id: "node-2".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() }, @@ -137,8 +158,12 @@ mod tests { assert!(table.contains_entry("test-index", "test-source")); let entry = table.find_entry("test-index", "test-source").unwrap(); - assert_eq!(entry.len(), 2); - assert_eq!(entry.shards()[0].shard_id, 0); - assert_eq!(entry.shards()[1].shard_id, 1); + assert_eq!(entry.shards.len(), 2); + assert_eq!(entry.shards[0].shard_id, 0); + assert_eq!(entry.shards[1].shard_id, 1); + + assert_eq!(entry.next_shard_round_robin().shard_id, 0); + assert_eq!(entry.next_shard_round_robin().shard_id, 1); + assert_eq!(entry.next_shard_round_robin().shard_id, 0); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs index 5e8186f1abe..07bc93ee0ea 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs @@ -22,8 +22,7 @@ use std::ops::RangeBounds; use mrecordlog::MultiRecordLog; use quickwit_proto::ingest::ShardState; -use super::ingester::{PrimaryShard, ReplicaShard}; -use crate::ingest_v2::ingester::Position; +use super::models::{Position, PrimaryShard, ReplicaShard}; pub(super) trait MultiRecordLogTestExt { fn assert_records_eq(&self, queue_id: &str, range: R, expected_records: &[(u64, &str)]) diff --git a/quickwit/quickwit-jaeger/src/integration_tests.rs b/quickwit/quickwit-jaeger/src/integration_tests.rs index d4163f918f0..f81c0d8bafd 100644 --- a/quickwit/quickwit-jaeger/src/integration_tests.rs +++ b/quickwit/quickwit-jaeger/src/integration_tests.rs @@ -386,9 +386,7 @@ async fn searcher_for_test( .grpc_advertise_addr; let searcher_client = SearchServiceClient::from_service(searcher_service.clone(), grpc_advertise_addr); - searcher_pool - .insert(grpc_advertise_addr, searcher_client) - .await; + searcher_pool.insert(grpc_advertise_addr, searcher_client); searcher_service } diff --git a/quickwit/quickwit-janitor/src/error.rs b/quickwit/quickwit-janitor/src/error.rs index c9581be10e5..612698b818d 100644 --- a/quickwit/quickwit-janitor/src/error.rs +++ b/quickwit/quickwit-janitor/src/error.rs @@ -35,11 +35,11 @@ pub enum JanitorError { } impl ServiceError for JanitorError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { JanitorError::InvalidDeleteQuery(_) => ServiceErrorCode::BadRequest, JanitorError::Internal(_) => ServiceErrorCode::Internal, - JanitorError::Metastore(error) => error.status_code(), + JanitorError::Metastore(error) => error.error_code(), } } } diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index d790ed9c8fe..c9532ce7871 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -238,6 +238,10 @@ impl Metastore for ControlPlaneMetastore { self.metastore.acquire_shards(request).await } + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { + self.metastore.list_shards(request).await + } + async fn close_shards( &self, request: CloseShardsRequest, @@ -245,10 +249,6 @@ impl Metastore for ControlPlaneMetastore { self.metastore.close_shards(request).await } - async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { - self.metastore.list_shards(request).await - } - async fn delete_shards( &self, request: DeleteShardsRequest, 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 92148ebcdf9..3694b420300 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 @@ -24,12 +24,11 @@ use std::fmt; use itertools::Either; use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ - AcquireShardsSubrequest, AcquireShardsSubresponse, CloseShardsFailure, CloseShardsSubrequest, - CloseShardsSuccess, DeleteShardsSubrequest, EntityKind, ListShardsSubrequest, - ListShardsSubresponse, MetastoreError, MetastoreResult, OpenShardsSubrequest, - OpenShardsSubresponse, + AcquireShardsSubrequest, AcquireShardsSubresponse, CloseShardsFailure, CloseShardsFailureKind, + CloseShardsSubrequest, CloseShardsSuccess, DeleteShardsSubrequest, EntityKind, + ListShardsSubrequest, ListShardsSubresponse, MetastoreError, MetastoreResult, + OpenShardsSubrequest, OpenShardsSubresponse, }; -use quickwit_proto::tonic::Code; use quickwit_proto::types::ShardId; use quickwit_proto::{queue_id, IndexUid, SourceId}; use serde::{Deserialize, Serialize}; @@ -146,7 +145,7 @@ impl Shards { shard_id=%shard.shard_id, leader_id=%shard.leader_id, follower_id=?shard.follower_id, - "Opened shard." + "opened shard" ); shard } @@ -211,8 +210,8 @@ impl Shards { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - error_code: Code::NotFound as u32, - error_message: "shard not found".to_string(), + failure_kind: CloseShardsFailureKind::NotFound as i32, + failure_message: "shard not found".to_string(), }; return Ok(MutationOccurred::No(Either::Right(failure))); }; @@ -235,20 +234,20 @@ impl Shards { index_id=%self.index_uid.index_id(), source_id=%shard.source_id, shard_id=%shard.shard_id, - "Closed shard.", + "closed shard", ); } other => { - let error_message = format!( - "Invalid `shard_state` argument: expected `Closing` or `Closed` state, got \ + let failure_message = format!( + "invalid `shard_state` argument: expected `Closing` or `Closed` state, got \ `{other:?}`.", ); let failure = CloseShardsFailure { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - error_code: Code::InvalidArgument as u32, - error_message, + failure_kind: CloseShardsFailureKind::InvalidArgument as i32, + failure_message, }; return Ok(MutationOccurred::No(Either::Right(failure))); } @@ -257,6 +256,9 @@ impl Shards { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, + leader_id: shard.leader_id.clone(), + follower_id: shard.follower_id.clone(), + publish_position_inclusive: shard.publish_position_inclusive.clone(), }; Ok(MutationOccurred::Yes(Either::Left(success))) } diff --git a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs index 0c1ffccd507..0dbae289b41 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs @@ -480,12 +480,12 @@ impl SpanStatus { } fn from_otlp(span_status: OtlpStatus) -> Self { - if span_status.code == OtlpStatusCode::Ok as i32 { + if span_status.code() == OtlpStatusCode::Ok { Self { code: OtlpStatusCode::Ok, message: None, } - } else if span_status.code == OtlpStatusCode::Error as i32 { + } else if span_status.code() == OtlpStatusCode::Error { let message = if span_status.message.is_empty() { None } else { diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 38d30d084f5..e73c63a5d52 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -25,11 +25,17 @@ import "quickwit/ingest.proto"; import "quickwit/metastore.proto"; service ControlPlaneService { - // The control plane acts as a proxy for the metastore for a subset of the API so it can track the state of the metastore accurately and react to events in real-time. + // The control plane acts as a proxy for the metastore for a subset of the API so it can track the state of the + // metastore accurately and react to events in real-time. - // Metastore index API proxy: + // The following RPCs are forwarded and handled by the metastore: // - `create_index` // - `delete_index` + // - `add_source` + // - `toggle_source` + // - `delete_source` + + // Index API // Creates a new index. rpc CreateIndex(quickwit.metastore.CreateIndexRequest) returns (quickwit.metastore.CreateIndexResponse); @@ -37,10 +43,7 @@ service ControlPlaneService { // Deletes an index. rpc DeleteIndex(quickwit.metastore.DeleteIndexRequest) returns (quickwit.metastore.EmptyResponse); - // Metastore source API proxy: - // - `add_source` - // - `toggle_source` - // - `delete_source` + // Source API // Adds a source to an index. rpc AddSource(quickwit.metastore.AddSourceRequest) returns (quickwit.metastore.EmptyResponse); @@ -55,9 +58,18 @@ service ControlPlaneService { // Returns the list of open shards for one or several sources. If the control plane is not able to find any // for a source, it will pick a pair of leader-follower ingesters and will open a new shard. - rpc GetOpenShards(GetOpenShardsRequest) returns (GetOpenShardsResponse); + rpc GetOrCreateOpenShards(GetOrCreateOpenShardsRequest) returns (GetOrCreateOpenShardsResponse); + + // The following RPCs are are metastore callbacks: + // - `close_shards` + // - `delete_shards` + // - `notify_index_change` + + // Closes a list of shards. This RPC is a metastore callback. + rpc CloseShards(quickwit.metastore.CloseShardsRequest) returns (quickwit.metastore.EmptyResponse); - rpc CloseShards(CloseShardsRequest) returns (CloseShardsResponse); + // Deletes a list of shards. This RPC is a metastore callback. + rpc DeleteShards(quickwit.metastore.DeleteShardsRequest) returns (quickwit.metastore.EmptyResponse); // Notify the Control Plane that a change on an index occurred. The change // can be an index creation, deletion, or update that includes a source creation/deletion/num pipeline update. @@ -75,18 +87,19 @@ message NotifyIndexChangeResponse {} // Shard API -message GetOpenShardsRequest { - repeated GetOpenShardsSubrequest subrequests = 1; +message GetOrCreateOpenShardsRequest { + repeated GetOrCreateOpenShardsSubrequest subrequests = 1; repeated string unavailable_ingesters = 2; } -message GetOpenShardsSubrequest { +message GetOrCreateOpenShardsSubrequest { string index_id = 1; string source_id = 2; + repeated uint64 closed_shards = 3; } // TODO: Handle partial failures. -message GetOpenShardsResponse { +message GetOrCreateOpenShardsResponse { repeated GetOpenShardsSubresponse subresponses = 1; } @@ -95,18 +108,3 @@ message GetOpenShardsSubresponse { string source_id = 2; repeated quickwit.ingest.Shard open_shards = 3; } - -message CloseShardsRequest { - repeated CloseShardsSubrequest subrequests = 1; -} - -message CloseShardsSubrequest { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - quickwit.ingest.ShardState shard_state = 4; - optional uint64 replication_position_inclusive = 5; -} - -message CloseShardsResponse { -} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 769d957da40..3ea4c9cf923 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -70,12 +70,15 @@ message PersistSuccess { optional uint64 replication_position_inclusive = 4; } +enum PersistFailureKind { + SHARD_CLOSED = 0; +} + message PersistFailure { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - // ingest.DocBatchV2 doc_batch = 4; - // ingest.IngestError error = 5; + PersistFailureKind failure_kind = 4; } message SynReplicationMessage { diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index d06ac7dd1a1..f7f33da2ed0 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -330,14 +330,22 @@ message CloseShardsSuccess { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; + string leader_id = 4; + optional string follower_id = 5; + string publish_position_inclusive = 6; +} + +enum CloseShardsFailureKind { + INVALID_ARGUMENT = 0; + NOT_FOUND = 1; } message CloseShardsFailure { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - uint32 error_code = 4; - string error_message = 5; + CloseShardsFailureKind failure_kind = 4; + string failure_message = 5; } message DeleteShardsRequest { 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 d51f4844745..b19a459efcf 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -9,26 +9,28 @@ pub struct NotifyIndexChangeResponse {} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct GetOpenShardsRequest { +pub struct GetOrCreateOpenShardsRequest { #[prost(message, repeated, tag = "1")] - pub subrequests: ::prost::alloc::vec::Vec, + pub subrequests: ::prost::alloc::vec::Vec, #[prost(string, repeated, tag = "2")] pub unavailable_ingesters: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct GetOpenShardsSubrequest { +pub struct GetOrCreateOpenShardsSubrequest { #[prost(string, tag = "1")] pub index_id: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, + #[prost(uint64, repeated, tag = "3")] + pub closed_shards: ::prost::alloc::vec::Vec, } /// TODO: Handle partial failures. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct GetOpenShardsResponse { +pub struct GetOrCreateOpenShardsResponse { #[prost(message, repeated, tag = "1")] pub subresponses: ::prost::alloc::vec::Vec, } @@ -43,32 +45,6 @@ pub struct GetOpenShardsSubresponse { #[prost(message, repeated, tag = "3")] pub open_shards: ::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 CloseShardsRequest { - #[prost(message, repeated, tag = "1")] - pub subrequests: ::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 CloseShardsSubrequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, - #[prost(enumeration = "super::ingest::ShardState", tag = "4")] - pub shard_state: i32, - #[prost(uint64, 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 CloseShardsResponse {} /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] @@ -101,14 +77,20 @@ pub trait ControlPlaneService: std::fmt::Debug + dyn_clone::DynClone + Send + Sy ) -> crate::control_plane::ControlPlaneResult; /// Returns the list of open shards for one or several sources. If the control plane is not able to find any /// for a source, it will pick a pair of leader-follower ingesters and will open a new shard. - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult; + request: GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult; + /// Closes a list of shards. This RPC is a metastore callback. async fn close_shards( &mut self, - request: CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult; + request: super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult; + /// Deletes a list of shards. This RPC is a metastore callback. + async fn delete_shards( + &mut self, + request: super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult; /// Notify the Control Plane that a change on an index occurred. The change /// can be an index creation, deletion, or update that includes a source creation/deletion/num pipeline update. /// Note(fmassot): it's not very clear for a user to know which change triggers a control plane notification. @@ -219,18 +201,24 @@ impl ControlPlaneService for ControlPlaneServiceClient { ) -> crate::control_plane::ControlPlaneResult { self.inner.delete_source(request).await } - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner.get_open_shards(request).await + request: GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.get_or_create_open_shards(request).await } async fn close_shards( &mut self, - request: CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.inner.close_shards(request).await } + async fn delete_shards( + &mut self, + request: super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.delete_shards(request).await + } async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, @@ -287,18 +275,30 @@ pub mod control_plane_service_mock { > { self.inner.lock().await.delete_source(request).await } - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: super::GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner.lock().await.get_open_shards(request).await + request: super::GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::GetOrCreateOpenShardsResponse, + > { + self.inner.lock().await.get_or_create_open_shards(request).await } async fn close_shards( &mut self, - request: super::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: super::super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::super::metastore::EmptyResponse, + > { self.inner.lock().await.close_shards(request).await } + async fn delete_shards( + &mut self, + request: super::super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::super::metastore::EmptyResponse, + > { + self.inner.lock().await.delete_shards(request).await + } async fn notify_index_change( &mut self, request: super::NotifyIndexChangeRequest, @@ -403,8 +403,8 @@ for Box { Box::pin(fut) } } -impl tower::Service for Box { - type Response = GetOpenShardsResponse; +impl tower::Service for Box { + type Response = GetOrCreateOpenShardsResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; fn poll_ready( @@ -413,14 +413,15 @@ impl tower::Service for Box { ) -> std::task::Poll> { std::task::Poll::Ready(Ok(())) } - fn call(&mut self, request: GetOpenShardsRequest) -> Self::Future { + fn call(&mut self, request: GetOrCreateOpenShardsRequest) -> Self::Future { let mut svc = self.clone(); - let fut = async move { svc.get_open_shards(request).await }; + let fut = async move { svc.get_or_create_open_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { - type Response = CloseShardsResponse; +impl tower::Service +for Box { + type Response = super::metastore::EmptyResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; fn poll_ready( @@ -429,12 +430,29 @@ impl tower::Service for Box { ) -> std::task::Poll> { std::task::Poll::Ready(Ok(())) } - fn call(&mut self, request: CloseShardsRequest) -> Self::Future { + fn call(&mut self, request: super::metastore::CloseShardsRequest) -> Self::Future { let mut svc = self.clone(); let fut = async move { svc.close_shards(request).await }; Box::pin(fut) } } +impl tower::Service +for Box { + type Response = super::metastore::EmptyResponse; + type Error = crate::control_plane::ControlPlaneError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: super::metastore::DeleteShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.delete_shards(request).await }; + Box::pin(fut) + } +} impl tower::Service for Box { type Response = NotifyIndexChangeResponse; type Error = crate::control_plane::ControlPlaneError; @@ -479,14 +497,19 @@ struct ControlPlaneServiceTowerBlock { super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, - get_open_shards_svc: quickwit_common::tower::BoxService< - GetOpenShardsRequest, - GetOpenShardsResponse, + get_or_create_open_shards_svc: quickwit_common::tower::BoxService< + GetOrCreateOpenShardsRequest, + GetOrCreateOpenShardsResponse, crate::control_plane::ControlPlaneError, >, close_shards_svc: quickwit_common::tower::BoxService< - CloseShardsRequest, - CloseShardsResponse, + super::metastore::CloseShardsRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + delete_shards_svc: quickwit_common::tower::BoxService< + super::metastore::DeleteShardsRequest, + super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, notify_index_change_svc: quickwit_common::tower::BoxService< @@ -503,8 +526,9 @@ impl Clone for ControlPlaneServiceTowerBlock { add_source_svc: self.add_source_svc.clone(), toggle_source_svc: self.toggle_source_svc.clone(), delete_source_svc: self.delete_source_svc.clone(), - get_open_shards_svc: self.get_open_shards_svc.clone(), + get_or_create_open_shards_svc: self.get_or_create_open_shards_svc.clone(), close_shards_svc: self.close_shards_svc.clone(), + delete_shards_svc: self.delete_shards_svc.clone(), notify_index_change_svc: self.notify_index_change_svc.clone(), } } @@ -543,18 +567,24 @@ impl ControlPlaneService for ControlPlaneServiceTowerBlock { ) -> crate::control_plane::ControlPlaneResult { self.delete_source_svc.ready().await?.call(request).await } - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.get_open_shards_svc.ready().await?.call(request).await + request: GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.get_or_create_open_shards_svc.ready().await?.call(request).await } async fn close_shards( &mut self, - request: CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.close_shards_svc.ready().await?.call(request).await } + async fn delete_shards( + &mut self, + request: super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.delete_shards_svc.ready().await?.call(request).await + } async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, @@ -610,11 +640,11 @@ pub struct ControlPlaneServiceTowerBlockBuilder { >, >, #[allow(clippy::type_complexity)] - get_open_shards_layer: Option< + get_or_create_open_shards_layer: Option< quickwit_common::tower::BoxLayer< Box, - GetOpenShardsRequest, - GetOpenShardsResponse, + GetOrCreateOpenShardsRequest, + GetOrCreateOpenShardsResponse, crate::control_plane::ControlPlaneError, >, >, @@ -622,8 +652,17 @@ pub struct ControlPlaneServiceTowerBlockBuilder { close_shards_layer: Option< quickwit_common::tower::BoxLayer< Box, - CloseShardsRequest, - CloseShardsResponse, + super::metastore::CloseShardsRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + >, + #[allow(clippy::type_complexity)] + delete_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + super::metastore::DeleteShardsRequest, + super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, >, @@ -682,17 +721,29 @@ impl ControlPlaneServiceTowerBlockBuilder { super::metastore::DeleteSourceRequest, >>::Future: Send + 'static, L::Service: tower::Service< - GetOpenShardsRequest, - Response = GetOpenShardsResponse, + GetOrCreateOpenShardsRequest, + Response = GetOrCreateOpenShardsResponse, Error = crate::control_plane::ControlPlaneError, > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, + >::Future: Send + 'static, L::Service: tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, + super::metastore::CloseShardsRequest, + Response = super::metastore::EmptyResponse, Error = crate::control_plane::ControlPlaneError, > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + super::metastore::DeleteShardsRequest, + Response = super::metastore::EmptyResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, L::Service: tower::Service< NotifyIndexChangeRequest, Response = NotifyIndexChangeResponse, @@ -721,13 +772,17 @@ impl ControlPlaneServiceTowerBlockBuilder { quickwit_common::tower::BoxLayer::new(layer.clone()), ); self - .get_open_shards_layer = Some( + .get_or_create_open_shards_layer = Some( quickwit_common::tower::BoxLayer::new(layer.clone()), ); self .close_shards_layer = Some( quickwit_common::tower::BoxLayer::new(layer.clone()), ); + self + .delete_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); self .notify_index_change_layer = Some( quickwit_common::tower::BoxLayer::new(layer), @@ -809,32 +864,54 @@ impl ControlPlaneServiceTowerBlockBuilder { self.delete_source_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); self } - pub fn get_open_shards_layer(mut self, layer: L) -> Self + pub fn get_or_create_open_shards_layer(mut self, layer: L) -> Self where L: tower::Layer> + Send + Sync + 'static, L::Service: tower::Service< - GetOpenShardsRequest, - Response = GetOpenShardsResponse, + GetOrCreateOpenShardsRequest, + Response = GetOrCreateOpenShardsResponse, Error = crate::control_plane::ControlPlaneError, > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, + >::Future: Send + 'static, { - self.get_open_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + .get_or_create_open_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); self } pub fn close_shards_layer(mut self, layer: L) -> Self where L: tower::Layer> + Send + Sync + 'static, L::Service: tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, + super::metastore::CloseShardsRequest, + Response = super::metastore::EmptyResponse, Error = crate::control_plane::ControlPlaneError, > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, + >::Future: Send + 'static, { self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn delete_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + super::metastore::DeleteShardsRequest, + Response = super::metastore::EmptyResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.delete_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn notify_index_change_layer(mut self, layer: L) -> Self where L: tower::Layer> + Send + Sync + 'static, @@ -914,7 +991,9 @@ impl ControlPlaneServiceTowerBlockBuilder { } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) }; - let get_open_shards_svc = if let Some(layer) = self.get_open_shards_layer { + let get_or_create_open_shards_svc = if let Some(layer) + = self.get_or_create_open_shards_layer + { layer.layer(boxed_instance.clone()) } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) @@ -924,6 +1003,11 @@ impl ControlPlaneServiceTowerBlockBuilder { } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) }; + let delete_shards_svc = if let Some(layer) = self.delete_shards_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; let notify_index_change_svc = if let Some(layer) = self.notify_index_change_layer { layer.layer(boxed_instance.clone()) @@ -936,8 +1020,9 @@ impl ControlPlaneServiceTowerBlockBuilder { add_source_svc, toggle_source_svc, delete_source_svc, - get_open_shards_svc, + get_or_create_open_shards_svc, close_shards_svc, + delete_shards_svc, notify_index_change_svc, }; ControlPlaneServiceClient::new(tower_block) @@ -1061,20 +1146,29 @@ where >, > + tower::Service< - GetOpenShardsRequest, - Response = GetOpenShardsResponse, + GetOrCreateOpenShardsRequest, + Response = GetOrCreateOpenShardsResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + GetOrCreateOpenShardsResponse, + crate::control_plane::ControlPlaneError, + >, + > + + tower::Service< + super::metastore::CloseShardsRequest, + Response = super::metastore::EmptyResponse, Error = crate::control_plane::ControlPlaneError, Future = BoxFuture< - GetOpenShardsResponse, + super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, > + tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, + super::metastore::DeleteShardsRequest, + Response = super::metastore::EmptyResponse, Error = crate::control_plane::ControlPlaneError, Future = BoxFuture< - CloseShardsResponse, + super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, > @@ -1120,16 +1214,22 @@ where ) -> crate::control_plane::ControlPlaneResult { self.call(request).await } - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.call(request).await } async fn close_shards( &mut self, - request: CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.call(request).await + } + async fn delete_shards( + &mut self, + request: super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.call(request).await } async fn notify_index_change( @@ -1213,26 +1313,36 @@ where .map(|response| response.into_inner()) .map_err(|error| error.into()) } - async fn get_open_shards( + async fn get_or_create_open_shards( &mut self, - request: GetOpenShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: GetOrCreateOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.inner - .get_open_shards(request) + .get_or_create_open_shards(request) .await .map(|response| response.into_inner()) .map_err(|error| error.into()) } async fn close_shards( &mut self, - request: CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { + request: super::metastore::CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.inner .close_shards(request) .await .map(|response| response.into_inner()) .map_err(|error| error.into()) } + async fn delete_shards( + &mut self, + request: super::metastore::DeleteShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .delete_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, @@ -1314,21 +1424,21 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } - async fn get_open_shards( + async fn get_or_create_open_shards( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { self.inner .clone() - .get_open_shards(request.into_inner()) + .get_or_create_open_shards(request.into_inner()) .await .map(tonic::Response::new) .map_err(|error| error.into()) } async fn close_shards( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { self.inner .clone() .close_shards(request.into_inner()) @@ -1336,6 +1446,17 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } + async fn delete_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .delete_shards(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } async fn notify_index_change( &self, request: tonic::Request, @@ -1596,11 +1717,11 @@ pub mod control_plane_service_grpc_client { } /// Returns the list of open shards for one or several sources. If the control plane is not able to find any /// for a source, it will pick a pair of leader-follower ingesters and will open a new shard. - pub async fn get_open_shards( + pub async fn get_or_create_open_shards( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -1614,23 +1735,24 @@ pub mod control_plane_service_grpc_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.control_plane.ControlPlaneService/GetOpenShards", + "/quickwit.control_plane.ControlPlaneService/GetOrCreateOpenShards", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.control_plane.ControlPlaneService", - "GetOpenShards", + "GetOrCreateOpenShards", ), ); self.inner.unary(req, path, codec).await } + /// Closes a list of shards. This RPC is a metastore callback. pub async fn close_shards( &mut self, - request: impl tonic::IntoRequest, + request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, > { self.inner @@ -1656,6 +1778,39 @@ pub mod control_plane_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Deletes a list of shards. This RPC is a metastore callback. + pub async fn delete_shards( + &mut self, + request: impl tonic::IntoRequest< + super::super::metastore::DeleteShardsRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.control_plane.ControlPlaneService/DeleteShards", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "DeleteShards", + ), + ); + self.inner.unary(req, path, codec).await + } /// Notify the Control Plane that a change on an index occurred. The change /// can be an index creation, deletion, or update that includes a source creation/deletion/num pipeline update. /// Note(fmassot): it's not very clear for a user to know which change triggers a control plane notification. @@ -1743,18 +1898,27 @@ pub mod control_plane_service_grpc_server { >; /// Returns the list of open shards for one or several sources. If the control plane is not able to find any /// for a source, it will pick a pair of leader-follower ingesters and will open a new shard. - async fn get_open_shards( + async fn get_or_create_open_shards( &self, - request: tonic::Request, + request: tonic::Request, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, >; + /// Closes a list of shards. This RPC is a metastore callback. async fn close_shards( &self, - request: tonic::Request, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// Deletes a list of shards. This RPC is a metastore callback. + async fn delete_shards( + &self, + request: tonic::Request, ) -> std::result::Result< - tonic::Response, + tonic::Response, tonic::Status, >; /// Notify the Control Plane that a change on an index occurred. The change @@ -2094,25 +2258,27 @@ pub mod control_plane_service_grpc_server { }; Box::pin(fut) } - "/quickwit.control_plane.ControlPlaneService/GetOpenShards" => { + "/quickwit.control_plane.ControlPlaneService/GetOrCreateOpenShards" => { #[allow(non_camel_case_types)] - struct GetOpenShardsSvc(pub Arc); + struct GetOrCreateOpenShardsSvc( + pub Arc, + ); impl< T: ControlPlaneServiceGrpc, - > tonic::server::UnaryService - for GetOpenShardsSvc { - type Response = super::GetOpenShardsResponse; + > tonic::server::UnaryService + for GetOrCreateOpenShardsSvc { + type Response = super::GetOrCreateOpenShardsResponse; type Future = BoxFuture< tonic::Response, tonic::Status, >; fn call( &mut self, - request: tonic::Request, + request: tonic::Request, ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { - (*inner).get_open_shards(request).await + (*inner).get_or_create_open_shards(request).await }; Box::pin(fut) } @@ -2124,7 +2290,7 @@ pub mod control_plane_service_grpc_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = GetOpenShardsSvc(inner); + let method = GetOrCreateOpenShardsSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( @@ -2145,16 +2311,19 @@ pub mod control_plane_service_grpc_server { struct CloseShardsSvc(pub Arc); impl< T: ControlPlaneServiceGrpc, - > tonic::server::UnaryService - for CloseShardsSvc { - type Response = super::CloseShardsResponse; + > tonic::server::UnaryService< + super::super::metastore::CloseShardsRequest, + > for CloseShardsSvc { + type Response = super::super::metastore::EmptyResponse; type Future = BoxFuture< tonic::Response, tonic::Status, >; fn call( &mut self, - request: tonic::Request, + request: tonic::Request< + super::super::metastore::CloseShardsRequest, + >, ) -> Self::Future { let inner = Arc::clone(&self.0); let fut = async move { @@ -2186,6 +2355,55 @@ pub mod control_plane_service_grpc_server { }; Box::pin(fut) } + "/quickwit.control_plane.ControlPlaneService/DeleteShards" => { + #[allow(non_camel_case_types)] + struct DeleteShardsSvc(pub Arc); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService< + super::super::metastore::DeleteShardsRequest, + > for DeleteShardsSvc { + type Response = super::super::metastore::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request< + super::super::metastore::DeleteShardsRequest, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).delete_shards(request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = DeleteShardsSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/quickwit.control_plane.ControlPlaneService/NotifyIndexChange" => { #[allow(non_camel_case_types)] struct NotifyIndexChangeSvc(pub Arc); 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 2551d47f25c..804431bf47c 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -56,10 +56,10 @@ pub struct PersistFailure { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - /// ingest.DocBatchV2 doc_batch = 4; - /// ingest.IngestError error = 5; #[prost(uint64, tag = "3")] pub shard_id: u64, + #[prost(enumeration = "PersistFailureKind", tag = "4")] + pub failure_kind: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -255,6 +255,31 @@ pub struct PingRequest { #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct PingResponse {} +#[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 PersistFailureKind { + ShardClosed = 0, +} +impl PersistFailureKind { + /// 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 { + PersistFailureKind::ShardClosed => "SHARD_CLOSED", + } + } + /// 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), + _ => None, + } + } +} /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; pub type IngesterServiceStream = quickwit_common::ServiceStream< diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 6dc8025db14..92cac9be663 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -364,6 +364,12 @@ pub struct CloseShardsSuccess { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, + #[prost(string, tag = "4")] + pub leader_id: ::prost::alloc::string::String, + #[prost(string, optional, tag = "5")] + pub follower_id: ::core::option::Option<::prost::alloc::string::String>, + #[prost(string, tag = "6")] + pub publish_position_inclusive: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -375,10 +381,10 @@ pub struct CloseShardsFailure { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint32, tag = "4")] - pub error_code: u32, + #[prost(enumeration = "CloseShardsFailureKind", tag = "4")] + pub failure_kind: i32, #[prost(string, tag = "5")] - pub error_message: ::prost::alloc::string::String, + pub failure_message: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -497,6 +503,34 @@ impl SourceType { } } } +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum CloseShardsFailureKind { + InvalidArgument = 0, + NotFound = 1, +} +impl CloseShardsFailureKind { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + CloseShardsFailureKind::InvalidArgument => "INVALID_ARGUMENT", + CloseShardsFailureKind::NotFound => "NOT_FOUND", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "INVALID_ARGUMENT" => Some(Self::InvalidArgument), + "NOT_FOUND" => Some(Self::NotFound), + _ => None, + } + } +} /// Generated client implementations. pub mod metastore_service_client { #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] diff --git a/quickwit/quickwit-proto/src/control_plane/mod.rs b/quickwit/quickwit-proto/src/control_plane/mod.rs index e69c9ccfb0b..793c344c3ea 100644 --- a/quickwit/quickwit-proto/src/control_plane/mod.rs +++ b/quickwit/quickwit-proto/src/control_plane/mod.rs @@ -27,7 +27,7 @@ include!("../codegen/quickwit/quickwit.control_plane.rs"); pub type ControlPlaneResult = std::result::Result; -#[derive(Debug, thiserror::Error)] +#[derive(Debug, thiserror::Error, Eq, PartialEq, Serialize, Deserialize)] pub enum ControlPlaneError { #[error("an internal error occurred: {0}")] Internal(String), @@ -52,22 +52,22 @@ impl From for MetastoreError { impl From for tonic::Status { fn from(error: ControlPlaneError) -> Self { - match error { - ControlPlaneError::Internal(message) => tonic::Status::internal(message), - ControlPlaneError::Metastore(error) => error.into(), - ControlPlaneError::Unavailable(message) => tonic::Status::unavailable(message), - } + 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) } } impl From for ControlPlaneError { fn from(status: tonic::Status) -> Self { - match status.code() { - tonic::Code::Unavailable => { - ControlPlaneError::Unavailable(status.message().to_string()) - } - _ => ControlPlaneError::Internal(status.message().to_string()), - } + serde_json::from_str(status.message()).unwrap_or_else(|_| { + ControlPlaneError::Internal(format!( + "failed to deserialize control plane error: `{}`", + status.message() + )) + }) } } @@ -86,10 +86,10 @@ impl From> for ControlPlaneError { } impl ServiceError for ControlPlaneError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { Self::Internal { .. } => ServiceErrorCode::Internal, - Self::Metastore(error) => error.status_code(), + Self::Metastore(error) => error.error_code(), Self::Unavailable(_) => ServiceErrorCode::Unavailable, } } diff --git a/quickwit/quickwit-proto/src/error.rs b/quickwit/quickwit-proto/src/error.rs index 67b6bfc0450..3743b85e805 100644 --- a/quickwit/quickwit-proto/src/error.rs +++ b/quickwit/quickwit-proto/src/error.rs @@ -68,16 +68,16 @@ impl ServiceErrorCode { pub trait ServiceError: ToString { fn grpc_error(&self) -> tonic::Status { - let grpc_code = self.status_code().to_grpc_status_code(); + let grpc_code = self.error_code().to_grpc_status_code(); let error_msg = self.to_string(); tonic::Status::new(grpc_code, error_msg) } - fn status_code(&self) -> ServiceErrorCode; + fn error_code(&self) -> ServiceErrorCode; } impl ServiceError for Infallible { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { unreachable!() } } diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index 266101d17ed..cd1bec40261 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -113,7 +113,7 @@ impl From for IndexingError { } impl ServiceError for IndexingError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { Self::MissingPipeline { .. } => ServiceErrorCode::NotFound, Self::PipelineAlreadyExists { .. } => ServiceErrorCode::BadRequest, diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index 647ff4f83e0..f7655b0a8fe 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -85,7 +85,7 @@ impl From for IngestV2Error { } impl ServiceError for IngestV2Error { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { Self::Internal { .. } => ServiceErrorCode::Internal, Self::IngesterUnavailable { .. } => ServiceErrorCode::Unavailable, diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index f2d142559a1..e630408fe7f 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -222,11 +222,11 @@ pub fn set_parent_span_from_request_metadata(request_metadata: &tonic::metadata: } impl ServiceError for quickwit_actors::AskError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { quickwit_actors::AskError::MessageNotDelivered => ServiceErrorCode::Internal, quickwit_actors::AskError::ProcessMessageError => ServiceErrorCode::Internal, - quickwit_actors::AskError::ErrorReply(err) => err.status_code(), + quickwit_actors::AskError::ErrorReply(err) => err.error_code(), } } } diff --git a/quickwit/quickwit-proto/src/metastore/events.rs b/quickwit/quickwit-proto/src/metastore/events.rs index b3ec894db10..a093e12a780 100644 --- a/quickwit/quickwit-proto/src/metastore/events.rs +++ b/quickwit/quickwit-proto/src/metastore/events.rs @@ -19,7 +19,7 @@ use quickwit_common::pubsub::Event; -use super::SourceType; +use super::{CloseShardsRequest, DeleteShardsRequest, SourceType}; use crate::{IndexUid, SourceId}; /// Create index event. @@ -71,3 +71,6 @@ impl Event for CreateIndexEvent {} impl Event for DeleteIndexEvent {} impl Event for DeleteSourceEvent {} impl Event for ToggleSourceEvent {} + +impl Event for CloseShardsRequest {} +impl Event for DeleteShardsRequest {} diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index 2e18e822811..2108afbabb6 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -22,7 +22,7 @@ use std::fmt; use quickwit_common::retry::Retryable; use serde::{Deserialize, Serialize}; -use crate::{IndexId, QueueId, ServiceError, ServiceErrorCode, SourceId, SplitId}; +use crate::{queue_id, IndexId, QueueId, ServiceError, ServiceErrorCode, SourceId, SplitId}; pub mod events; @@ -153,7 +153,7 @@ impl From for MetastoreError { impl From for tonic::Status { fn from(metastore_error: MetastoreError) -> Self { - let grpc_code = metastore_error.status_code().to_grpc_status_code(); + 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) @@ -161,9 +161,9 @@ impl From for tonic::Status { } impl ServiceError for MetastoreError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { - Self::AlreadyExists { .. } => ServiceErrorCode::BadRequest, + Self::AlreadyExists { .. } => ServiceErrorCode::AlreadyExists, Self::Connection { .. } => ServiceErrorCode::Internal, Self::Db { .. } => ServiceErrorCode::Internal, Self::FailedPrecondition { .. } => ServiceErrorCode::BadRequest, @@ -209,6 +209,18 @@ impl SourceType { } } +impl CloseShardsSuccess { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +impl CloseShardsFailure { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + pub mod serde_utils { use serde::{Deserialize, Serialize}; diff --git a/quickwit/quickwit-proto/src/types.rs b/quickwit/quickwit-proto/src/types.rs index dc943611f46..0ff71123fbe 100644 --- a/quickwit/quickwit-proto/src/types.rs +++ b/quickwit/quickwit-proto/src/types.rs @@ -45,6 +45,14 @@ pub fn queue_id(index_uid: &str, source_id: &str, shard_id: u64) -> QueueId { format!("{}/{}/{}", index_uid, source_id, shard_id) } +pub fn split_queue_id(queue_id: &str) -> Option<(IndexUid, SourceId, ShardId)> { + let mut parts = queue_id.split('/'); + let index_uid = parts.next()?; + let source_id = parts.next()?; + let shard_id = parts.next()?.parse::().ok()?; + Some((index_uid.into(), source_id.to_string(), shard_id)) +} + /// 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. @@ -112,6 +120,12 @@ impl From for IndexUid { } } +impl PartialEq<&str> for IndexUid { + fn eq(&self, other: &&str) -> bool { + self.as_str() == *other + } +} + #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)] pub struct NodeId(String); @@ -293,6 +307,32 @@ impl ToOwned for NodeIdRef { mod tests { use super::*; + #[test] + fn test_queue_id() { + assert_eq!( + queue_id("test-index:0", "test-source", 1), + "test-index:0/test-source/1" + ); + } + + #[test] + fn test_split_queue_id() { + let splits = split_queue_id("test-index:0"); + assert!(splits.is_none()); + + let splits = split_queue_id("test-index:0/test-source"); + assert!(splits.is_none()); + + let splits = split_queue_id("test-index:0/test-source/a"); + assert!(splits.is_none()); + + let (index_uid, source_id, shard_id) = + split_queue_id("test-index:0/test-source/1").unwrap(); + assert_eq!(index_uid, "test-index:0"); + assert_eq!(source_id, "test-source"); + assert_eq!(shard_id, 1); + } + #[test] fn test_node_id() { let node_id = NodeId::new("test-node".to_string()); diff --git a/quickwit/quickwit-query/src/query_ast/field_presence.rs b/quickwit/quickwit-query/src/query_ast/field_presence.rs index 153f5be2794..be68c6a8d38 100644 --- a/quickwit/quickwit-query/src/query_ast/field_presence.rs +++ b/quickwit/quickwit-query/src/query_ast/field_presence.rs @@ -129,7 +129,7 @@ mod tests { #[test] fn test_field_presence_hash_escaped_dot() { let field_presence_term: u64 = - compute_field_presence_hash(Field::from_field_id(17u32), r#"attributes\.color.hello"#); + compute_field_presence_hash(Field::from_field_id(17u32), r"attributes\.color.hello"); assert_eq!( field_presence_term, PathHasher::hash_path(&[&17u32.to_le_bytes()[..], b"attributes.color", b"hello"]) diff --git a/quickwit/quickwit-search/src/cluster_client.rs b/quickwit/quickwit-search/src/cluster_client.rs index cf5500d6b45..87f7eb1f4a3 100644 --- a/quickwit/quickwit-search/src/cluster_client.rs +++ b/quickwit/quickwit-search/src/cluster_client.rs @@ -483,7 +483,7 @@ mod tests { ("127.0.0.1:1002", mock_search_service_2), ]); let first_client_addr: SocketAddr = "127.0.0.1:1001".parse().unwrap(); - let first_client = searcher_pool.get(&first_client_addr).await.unwrap(); + let first_client = searcher_pool.get(&first_client_addr).unwrap(); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer); let fetch_docs_response = cluster_client @@ -504,7 +504,7 @@ mod tests { ); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); let first_client_addr: SocketAddr = "127.0.0.1:1001".parse().unwrap(); - let first_client = searcher_pool.get(&first_client_addr).await.unwrap(); + let first_client = searcher_pool.get(&first_client_addr).unwrap(); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer); let search_error = cluster_client @@ -688,7 +688,7 @@ mod tests { drop(result_sender); let first_client_addr: SocketAddr = "127.0.0.1:1001".parse().unwrap(); - let first_client = searcher_pool.get(&first_client_addr).await.unwrap(); + let first_client = searcher_pool.get(&first_client_addr).unwrap(); let result = cluster_client .leaf_search_stream(request, first_client) .await; diff --git a/quickwit/quickwit-search/src/error.rs b/quickwit/quickwit-search/src/error.rs index 4865eb4ca75..14416e1e728 100644 --- a/quickwit/quickwit-search/src/error.rs +++ b/quickwit/quickwit-search/src/error.rs @@ -45,7 +45,7 @@ pub enum SearchError { } impl ServiceError for SearchError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { SearchError::IndexesNotFound { .. } => ServiceErrorCode::NotFound, SearchError::Internal(_) => ServiceErrorCode::Internal, diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 5ec4fa7c701..a31dd041aea 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -202,9 +202,7 @@ pub async fn single_node_search( )); let search_service_client = SearchServiceClient::from_service(search_service.clone(), socket_addr); - searcher_pool - .insert(socket_addr, search_service_client) - .await; + searcher_pool.insert(socket_addr, search_service_client); root_search( &searcher_context, search_request, diff --git a/quickwit/quickwit-search/src/search_job_placer.rs b/quickwit/quickwit-search/src/search_job_placer.rs index 648daa06b16..ebabfc2acf4 100644 --- a/quickwit/quickwit-search/src/search_job_placer.rs +++ b/quickwit/quickwit-search/src/search_job_placer.rs @@ -67,7 +67,7 @@ pub struct SearchJobPlacer { impl EventSubscriber for SearchJobPlacer { async fn handle_event(&mut self, evt: ReportSplitsRequest) { let mut nodes: HashMap = - self.searcher_pool.all().await.into_iter().collect(); + self.searcher_pool.pairs().collect(); if nodes.is_empty() { return; } @@ -127,9 +127,7 @@ impl SearchJobPlacer { ) -> impl Iterator { let mut nodes: Vec = self .searcher_pool - .all() - .await - .into_iter() + .pairs() .map(|(socket_addr, client)| SocketAddrAndClient { socket_addr, client, @@ -150,13 +148,11 @@ impl SearchJobPlacer { mut jobs: Vec, excluded_addrs: &HashSet, ) -> anyhow::Result)>> { - let num_nodes = self.searcher_pool.len().await; + let num_nodes = self.searcher_pool.len(); let mut candidate_nodes: Vec = self .searcher_pool - .all() - .await - .into_iter() + .pairs() .filter(|(grpc_addr, _)| { excluded_addrs.is_empty() || excluded_addrs.len() == num_nodes diff --git a/quickwit/quickwit-search/src/search_stream/leaf.rs b/quickwit/quickwit-search/src/search_stream/leaf.rs index 466889497df..e4d73d6cd89 100644 --- a/quickwit/quickwit-search/src/search_stream/leaf.rs +++ b/quickwit/quickwit-search/src/search_stream/leaf.rs @@ -673,7 +673,7 @@ mod tests { let test_sandbox = TestSandbox::create(index_id, doc_mapping_yaml, "", &["body"]).await?; let mut docs = Vec::new(); - let partition_by_fast_field_values = vec![1, 2, 3, 4, 5]; + let partition_by_fast_field_values = [1, 2, 3, 4, 5]; let mut expected_output_tmp: HashMap> = HashMap::new(); let start_timestamp = 72057595; let end_timestamp: i64 = start_timestamp + 20; diff --git a/quickwit/quickwit-search/src/tests.rs b/quickwit/quickwit-search/src/tests.rs index 3edf04c31ef..0ae05eb294e 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -1114,7 +1114,7 @@ async fn test_search_dynamic_mode_expand_dots() -> anyhow::Result<()> { assert_eq!(&docs[..], &[0u32]); } { - let docs = test_search_util(&test_sandbox, r#"k8s\.component\.name:quickwit"#).await; + let docs = test_search_util(&test_sandbox, r"k8s\.component\.name:quickwit").await; assert_eq!(&docs[..], &[0u32]); } test_sandbox.assert_quit().await; @@ -1140,7 +1140,7 @@ async fn test_search_dynamic_mode_do_not_expand_dots() -> anyhow::Result<()> { let docs = vec![json!({"k8s.component.name": "quickwit"})]; test_sandbox.add_documents(docs).await.unwrap(); { - let docs = test_search_util(&test_sandbox, r#"k8s\.component\.name:quickwit"#).await; + let docs = test_search_util(&test_sandbox, r"k8s\.component\.name:quickwit").await; assert_eq!(&docs[..], &[0u32]); } { diff --git a/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs b/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs index a0c0ba4390b..4efc6426f05 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/bulk.rs @@ -46,11 +46,11 @@ pub enum IngestRestApiError { } impl ServiceError for IngestRestApiError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { match self { Self::BulkInvalidAction(_) => ServiceErrorCode::BadRequest, Self::BulkInvalidSource(_) => ServiceErrorCode::BadRequest, - Self::IngestApi(ingest_api_error) => ingest_api_error.status_code(), + Self::IngestApi(ingest_api_error) => ingest_api_error.error_code(), } } } diff --git a/quickwit/quickwit-serve/src/elastic_search_api/model/error.rs b/quickwit/quickwit-serve/src/elastic_search_api/model/error.rs index 5fe1f55f45e..95ebf81a336 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/model/error.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/model/error.rs @@ -32,7 +32,7 @@ pub struct ElasticSearchError { impl From for ElasticSearchError { fn from(search_error: SearchError) -> Self { - let status = search_error.status_code().to_http_status_code(); + let status = search_error.error_code().to_http_status_code(); // Fill only reason field to keep it simple. let reason = ErrorCause { reason: Some(search_error.to_string()), diff --git a/quickwit/quickwit-serve/src/ingest_metastore.rs b/quickwit/quickwit-serve/src/ingest_metastore.rs new file mode 100644 index 00000000000..1dc9e7110b6 --- /dev/null +++ b/quickwit/quickwit-serve/src/ingest_metastore.rs @@ -0,0 +1,65 @@ +// Copyright (C) 2023 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::sync::Arc; + +use async_trait::async_trait; +use quickwit_metastore::Metastore; +use quickwit_proto::ingest::IngestV2Error; +use quickwit_proto::metastore::{ + CloseShardsRequest, CloseShardsResponse, DeleteShardsRequest, DeleteShardsResponse, +}; + +// TODO: Remove when the metastore is code generated in `quickwit-proto`. + +/// Implementation of the [`quickwit_ingest::IngestMetastore`] trait. See comment in the module +/// where it is defined for more details about why this is required. +#[derive(Clone)] +pub(crate) struct IngestMetastore { + metastore: Arc, +} + +impl IngestMetastore { + pub fn new(metastore: Arc) -> Self { + Self { metastore } + } +} + +#[async_trait] +impl quickwit_ingest::IngestMetastore for IngestMetastore { + async fn close_shards( + &self, + request: CloseShardsRequest, + ) -> quickwit_proto::ingest::IngestV2Result { + self.metastore + .close_shards(request) + .await + .map_err(|error| IngestV2Error::Internal(error.to_string())) + } + + async fn delete_shards( + &self, + request: DeleteShardsRequest, + ) -> quickwit_proto::ingest::IngestV2Result { + self.metastore + .delete_shards(request) + .await + .map_err(|error| IngestV2Error::Internal(error.to_string())) + } +} diff --git a/quickwit/quickwit-serve/src/json_api_response.rs b/quickwit/quickwit-serve/src/json_api_response.rs index 071dc6fdd1f..b175861ec79 100644 --- a/quickwit/quickwit-serve/src/json_api_response.rs +++ b/quickwit/quickwit-serve/src/json_api_response.rs @@ -38,7 +38,7 @@ pub(crate) struct ApiError { } impl ServiceError for ApiError { - fn status_code(&self) -> ServiceErrorCode { + fn error_code(&self) -> ServiceErrorCode { self.service_code } } @@ -57,12 +57,12 @@ pub(crate) fn make_json_api_response( format: BodyFormat, ) -> JsonApiResponse { let result_with_api_error = result.map_err(|err| ApiError { - service_code: err.status_code(), + service_code: err.error_code(), message: err.to_string(), }); let status_code = match &result_with_api_error { Ok(_) => status::StatusCode::OK, - Err(err) => err.status_code().to_http_status_code(), + Err(err) => err.error_code().to_http_status_code(), }; JsonApiResponse::new(&result_with_api_error, status_code, &format) } diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 89a60103880..edfaa2fed18 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -27,6 +27,7 @@ mod health_check_api; mod index_api; mod indexing_api; mod ingest_api; +mod ingest_metastore; mod json_api_response; mod metrics; mod metrics_api; @@ -51,6 +52,7 @@ use anyhow::Context; use byte_unit::n_mib_bytes; pub use format::BodyFormat; use futures::{Stream, StreamExt}; +use ingest_metastore::IngestMetastore; use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox, Universe}; use quickwit_cluster::{start_cluster_service, Cluster, ClusterChange, ClusterMember}; @@ -361,6 +363,7 @@ pub async fn serve_quickwit( let (ingest_router_service, ingester_service_opt) = setup_ingest_v2( &node_config, &cluster, + metastore_client.clone(), control_plane_service.clone(), ingester_pool, ) @@ -581,6 +584,7 @@ fn setup_control_plane_event_subscriptions( async fn setup_ingest_v2( config: &NodeConfig, cluster: &Cluster, + metastore: Arc, control_plane: ControlPlaneServiceClient, ingester_pool: IngesterPool, ) -> anyhow::Result<(IngestRouterServiceClient, Option)> { @@ -606,6 +610,7 @@ async fn setup_ingest_v2( let ingester = Ingester::try_new( self_node_id.clone(), + Arc::new(IngestMetastore::new(metastore)), ingester_pool.clone(), &wal_dir_path, replication_factor, @@ -961,9 +966,9 @@ mod tests { .unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert_eq!(indexer_pool.len().await, 1); + assert_eq!(indexer_pool.len(), 1); - let new_indexer_node_info = indexer_pool.get("test-indexer-node").await.unwrap(); + let new_indexer_node_info = indexer_pool.get("test-indexer-node").unwrap(); assert!(new_indexer_node_info.indexing_tasks.is_empty()); let new_indexing_task = IndexingTask { @@ -985,7 +990,7 @@ mod tests { .unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - let updated_indexer_node_info = indexer_pool.get("test-indexer-node").await.unwrap(); + let updated_indexer_node_info = indexer_pool.get("test-indexer-node").unwrap(); assert_eq!(updated_indexer_node_info.indexing_tasks.len(), 1); assert_eq!( updated_indexer_node_info.indexing_tasks[0], @@ -998,7 +1003,7 @@ mod tests { .unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; - assert!(indexer_pool.is_empty().await); + assert!(indexer_pool.is_empty()); } #[tokio::test] diff --git a/quickwit/quickwit-serve/src/search_api/mod.rs b/quickwit/quickwit-serve/src/search_api/mod.rs index 60b1bf0cf49..be368b94913 100644 --- a/quickwit/quickwit-serve/src/search_api/mod.rs +++ b/quickwit/quickwit-serve/src/search_api/mod.rs @@ -122,9 +122,7 @@ mod tests { start_test_server(grpc_addr, Arc::new(mock_search_service)).await?; let searcher_pool = SearcherPool::default(); - searcher_pool - .insert(grpc_addr, create_search_client_from_grpc_addr(grpc_addr)) - .await; + searcher_pool.insert(grpc_addr, create_search_client_from_grpc_addr(grpc_addr)); let search_job_placer = SearchJobPlacer::new(searcher_pool); let cluster_client = ClusterClient::new(search_job_placer.clone()); let stream = root_search_stream(request, &metastore, cluster_client).await?; diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index d0d27b78d93..0ca8b9a8edc 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -446,7 +446,7 @@ fn make_streaming_reply(result: Result) -> impl Reply warp::reply::Response::new(body) } Err(err) => { - status_code = err.status_code().to_http_status_code(); + status_code = err.error_code().to_http_status_code(); warp::reply::Response::new(hyper::Body::from(err.to_string())) } }; diff --git a/quickwit/quickwit-serve/src/ui_handler.rs b/quickwit/quickwit-serve/src/ui_handler.rs index 9080856d6be..0d4585f91cd 100644 --- a/quickwit/quickwit-serve/src/ui_handler.rs +++ b/quickwit/quickwit-serve/src/ui_handler.rs @@ -28,7 +28,7 @@ use warp::{Filter, Rejection}; /// Regular expression to identify which path should serve an asset file. /// If not matched, the server serves the `index.html` file. -const PATH_PATTERN: &str = r#"(^static|\.(png|json|txt|ico|js|map)$)"#; +const PATH_PATTERN: &str = r"(^static|\.(png|json|txt|ico|js|map)$)"; const UI_INDEX_FILE_NAME: &str = "index.html"; diff --git a/quickwit/quickwit-storage/src/local_file_storage.rs b/quickwit/quickwit-storage/src/local_file_storage.rs index 709bf958deb..e18653a6fdc 100644 --- a/quickwit/quickwit-storage/src/local_file_storage.rs +++ b/quickwit/quickwit-storage/src/local_file_storage.rs @@ -392,7 +392,7 @@ mod tests { let temp_dir = tempfile::tempdir()?; let index_uri = Uri::from_well_formed(format!("file://{}/foo/bar", temp_dir.path().display())); - let local_file_storage_factory = LocalFileStorageFactory::default(); + 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); diff --git a/quickwit/rust-toolchain.toml b/quickwit/rust-toolchain.toml index 9bb886815cd..7a07931dc29 100644 --- a/quickwit/rust-toolchain.toml +++ b/quickwit/rust-toolchain.toml @@ -1,4 +1,4 @@ [toolchain] -channel = "1.70" +channel = "1.72" components = ["cargo", "clippy", "rustfmt", "rust-docs"]