diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 324e0d99da7..0f34a3017e6 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -45,6 +45,7 @@ use quickwit_indexing::models::{ use quickwit_ingest::IngesterPool; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::CpuCapacity; +use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::search::{CountHits, SearchResponse}; use quickwit_proto::types::{IndexId, PipelineUid, SourceId, SplitId}; @@ -936,8 +937,9 @@ async fn create_empty_cluster(config: &NodeConfig) -> anyhow::Result { enabled_services: HashSet::new(), gossip_advertise_addr: config.gossip_advertise_addr, grpc_advertise_addr: config.grpc_advertise_addr, - indexing_cpu_capacity: CpuCapacity::zero(), indexing_tasks: Vec::new(), + indexing_cpu_capacity: CpuCapacity::zero(), + ingester_status: IngesterStatus::default(), availability_zone: None, }; let client_grpc_config = make_client_grpc_config(&config.grpc_config)?; diff --git a/quickwit/quickwit-cluster/src/change.rs b/quickwit/quickwit-cluster/src/change.rs index 1491212281c..77d3165f5f5 100644 --- a/quickwit/quickwit-cluster/src/change.rs +++ b/quickwit/quickwit-cluster/src/change.rs @@ -35,7 +35,10 @@ use crate::member::NodeStateExt; #[derive(Debug, Clone)] pub enum ClusterChange { Add(ClusterNode), - Update(ClusterNode), + Update { + previous: ClusterNode, + updated: ClusterNode, + }, Remove(ClusterNode), } @@ -246,7 +249,10 @@ async fn compute_cluster_change_events_on_updated( ); Some(ClusterChange::Remove(updated_node)) } else if previous_node.is_ready() && updated_node.is_ready() { - Some(ClusterChange::Update(updated_node)) + Some(ClusterChange::Update { + previous: previous_node, + updated: updated_node, + }) } else { None } @@ -681,16 +687,16 @@ pub(crate) mod tests { .await .unwrap(); - let ClusterChange::Update(node) = event else { + let ClusterChange::Update { updated, .. } = event else { panic!("expected `ClusterChange::Remove` event, got `{event:?}`"); }; - assert_eq!(node.chitchat_id(), &updated_chitchat_id); - assert_eq!(node.grpc_advertise_addr(), grpc_advertise_addr); - assert!(!node.is_self_node()); - assert!(node.is_ready()); + assert_eq!(updated.chitchat_id(), &updated_chitchat_id); + assert_eq!(updated.grpc_advertise_addr(), grpc_advertise_addr); + assert!(!updated.is_self_node()); + assert!(updated.is_ready()); assert_eq!( previous_nodes.get(&updated_chitchat_id.node_id).unwrap(), - &node + &updated ); } { @@ -1009,7 +1015,7 @@ pub(crate) mod tests { .await; assert_eq!(events.len(), 1); - let ClusterChange::Update(_node) = events[0].clone() else { + let ClusterChange::Update { .. } = events[0].clone() else { panic!( "Expected `ClusterChange::Update` event, got `{:?}`", events[0] diff --git a/quickwit/quickwit-cluster/src/cluster.rs b/quickwit/quickwit-cluster/src/cluster.rs index e3244746e0a..0ba54f7f71f 100644 --- a/quickwit/quickwit-cluster/src/cluster.rs +++ b/quickwit/quickwit-cluster/src/cluster.rs @@ -697,6 +697,7 @@ pub async fn create_cluster_for_test_with_id( self_node_readiness: bool, ) -> anyhow::Result { use quickwit_proto::indexing::PIPELINE_FULL_CAPACITY; + use quickwit_proto::ingest::ingester::IngesterStatus; let gossip_advertise_addr: SocketAddr = ([127, 0, 0, 1], gossip_advertise_port).into(); let self_node = ClusterMember { node_id, @@ -707,6 +708,7 @@ pub async fn create_cluster_for_test_with_id( grpc_advertise_addr: grpc_addr_from_listen_addr_for_test(gossip_advertise_addr), indexing_tasks: Vec::new(), indexing_cpu_capacity: PIPELINE_FULL_CAPACITY, + ingester_status: IngesterStatus::default(), availability_zone: None, }; let failure_detector_config = create_failure_detector_config_for_test(); diff --git a/quickwit/quickwit-cluster/src/lib.rs b/quickwit/quickwit-cluster/src/lib.rs index 30bb03e7f02..0f2dbebf749 100644 --- a/quickwit/quickwit-cluster/src/lib.rs +++ b/quickwit/quickwit-cluster/src/lib.rs @@ -36,6 +36,7 @@ use quickwit_common::tower::ClientGrpcConfig; use quickwit_config::service::QuickwitService; use quickwit_config::{GrpcConfig, NodeConfig, TlsConfig}; use quickwit_proto::indexing::CpuCapacity; +use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::tonic::transport::{Certificate, ClientTlsConfig, Identity}; use time::OffsetDateTime; @@ -143,6 +144,7 @@ pub async fn start_cluster_service(node_config: &NodeConfig) -> anyhow::Result usize; + fn ingester_status(&self) -> IngesterStatus; + fn availability_zone(&self) -> Option; } @@ -79,6 +83,12 @@ impl NodeStateExt for NodeState { .sum() } + fn ingester_status(&self) -> IngesterStatus { + self.get(INGESTER_STATUS_KEY) + .and_then(IngesterStatus::from_json_str_name) + .unwrap_or_default() + } + fn availability_zone(&self) -> Option { self.get(AVAILABILITY_ZONE_KEY).map(|az| az.to_string()) } @@ -108,6 +118,10 @@ pub struct ClusterMember { pub indexing_tasks: Vec, /// Indexing cpu capacity of the node expressed in milli cpu. pub indexing_cpu_capacity: CpuCapacity, + /// Status of the ingester service running on the node. `IngesterStatus::Unspecified` if the + /// node is not an ingester. + pub ingester_status: IngesterStatus, + /// Whether the node is ready to serve requests. pub is_ready: bool, /// Availability zone the node is running in, if enabled. pub availability_zone: Option, @@ -159,10 +173,12 @@ pub(crate) fn build_cluster_member( .map(|enabled_services_str| { parse_enabled_services_str(enabled_services_str, &chitchat_id.node_id) })?; - let availability_zone = node_state.availability_zone(); let grpc_advertise_addr = node_state.grpc_advertise_addr()?; let indexing_tasks = parse_indexing_tasks(node_state); let indexing_cpu_capacity = parse_indexing_cpu_capacity(node_state); + let ingester_status = node_state.ingester_status(); + let availability_zone = node_state.availability_zone(); + let member = ClusterMember { node_id: chitchat_id.node_id.into(), generation_id: chitchat_id.generation_id.into(), @@ -172,6 +188,7 @@ pub(crate) fn build_cluster_member( grpc_advertise_addr, indexing_tasks, indexing_cpu_capacity, + ingester_status, availability_zone, }; Ok(member) diff --git a/quickwit/quickwit-cluster/src/node.rs b/quickwit/quickwit-cluster/src/node.rs index 4a8b11dbafc..e47719cb4db 100644 --- a/quickwit/quickwit-cluster/src/node.rs +++ b/quickwit/quickwit-cluster/src/node.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use chitchat::{ChitchatId, NodeState}; use quickwit_config::service::QuickwitService; use quickwit_proto::indexing::{CpuCapacity, IndexingTask}; +use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::types::NodeIdRef; use tonic::transport::Channel; @@ -46,6 +47,7 @@ impl ClusterNode { grpc_advertise_addr: member.grpc_advertise_addr, indexing_tasks: member.indexing_tasks, indexing_capacity: member.indexing_cpu_capacity, + ingester_status: member.ingester_status, is_ready: member.is_ready, is_self_node, }; @@ -63,6 +65,27 @@ impl ClusterNode { enabled_services: &[&str], indexing_tasks: &[IndexingTask], ) -> Self { + Self::for_test_with_ingester_status( + node_id, + port, + is_self_node, + enabled_services, + indexing_tasks, + IngesterStatus::default(), + ) + .await + } + + #[cfg(any(test, feature = "testsuite"))] + pub async fn for_test_with_ingester_status( + node_id: &str, + port: u16, + is_self_node: bool, + enabled_services: &[&str], + indexing_tasks: &[IndexingTask], + ingester_status: IngesterStatus, + ) -> Self { + use quickwit_common::shared_consts::INGESTER_STATUS_KEY; use quickwit_common::tower::{ClientGrpcConfig, make_channel}; use crate::cluster::set_indexing_tasks_in_node_state; @@ -75,6 +98,7 @@ impl ClusterNode { let mut node_state = NodeState::for_test(); node_state.set(ENABLED_SERVICES_KEY, enabled_services.join(",")); node_state.set(GRPC_ADVERTISE_ADDR_KEY, grpc_advertise_addr.to_string()); + node_state.set(INGESTER_STATUS_KEY, ingester_status.as_json_str_name()); set_indexing_tasks_in_node_state(indexing_tasks, &mut node_state); Self::try_new(chitchat_id, &node_state, channel, is_self_node).unwrap() } @@ -125,6 +149,10 @@ impl ClusterNode { self.inner.indexing_capacity } + pub fn ingester_status(&self) -> IngesterStatus { + self.inner.ingester_status + } + pub fn is_ready(&self) -> bool { self.inner.is_ready } @@ -163,6 +191,7 @@ struct InnerNode { grpc_advertise_addr: SocketAddr, indexing_tasks: Vec, indexing_capacity: CpuCapacity, + ingester_status: IngesterStatus, is_ready: bool, is_self_node: bool, } diff --git a/quickwit/quickwit-common/src/shared_consts.rs b/quickwit/quickwit-common/src/shared_consts.rs index 9923705f0b2..d46cbc535ca 100644 --- a/quickwit/quickwit-common/src/shared_consts.rs +++ b/quickwit/quickwit-common/src/shared_consts.rs @@ -61,9 +61,12 @@ pub fn split_deletion_grace_period() -> Duration { /// being requested. pub const SCROLL_BATCH_LEN: usize = 1_000; -/// Prefix used in chitchat to broadcast the list of primary shards hosted by a leader. +/// Key prefix used in chitchat to broadcast the list of primary shards hosted by a leader. pub const INGESTER_PRIMARY_SHARDS_PREFIX: &str = "ingester.primary_shards:"; +/// Key used in chitchat to broadcast the status of an ingester. +pub const INGESTER_STATUS_KEY: &str = "ingester.status"; + /// File name for the encoded list of fields in the split pub const SPLIT_FIELDS_FILE_NAME: &str = "split_fields"; diff --git a/quickwit/quickwit-common/src/tower/pool.rs b/quickwit/quickwit-common/src/tower/pool.rs index 87a6f15e879..cfc1de93249 100644 --- a/quickwit/quickwit-common/src/tower/pool.rs +++ b/quickwit/quickwit-common/src/tower/pool.rs @@ -109,6 +109,16 @@ where .collect() } + /// Returns all the key-value pairs in the pool. + pub fn keys_values(&self) -> Vec<(K, V)> { + self.pool + .read() + .expect("lock should not be poisoned") + .iter() + .map(|(key, value)| (key.clone(), value.clone())) + .collect() + } + /// Returns all the values in the pool. pub fn values(&self) -> Vec { self.pool diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index e4c6995d639..0839bfecd22 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::btree_map::Entry; use std::collections::{BTreeMap, BTreeSet}; use std::fmt; use std::fmt::Formatter; @@ -27,14 +28,11 @@ use quickwit_actors::{ Actor, ActorContext, ActorExitStatus, ActorHandle, DeferableReplyHandler, Handler, Mailbox, Supervisor, Universe, WeakMailbox, }; -use quickwit_cluster::{ - ClusterChange, ClusterChangeStream, ClusterChangeStreamFactory, ClusterNode, -}; +use quickwit_cluster::{ClusterChange, ClusterChangeStream, ClusterChangeStreamFactory}; use quickwit_common::pretty::PrettyDisplay; use quickwit_common::pubsub::EventSubscriber; use quickwit_common::uri::Uri; use quickwit_common::{Progress, shared_consts}; -use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IndexConfig, IndexTemplate, SourceConfig}; use quickwit_ingest::{IngesterPool, LocalShardsUpdate}; use quickwit_metastore::{CreateIndexRequestExt, CreateIndexResponseExt, IndexMetadataResponseExt}; @@ -43,6 +41,7 @@ use quickwit_proto::control_plane::{ GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, GetOrCreateOpenShardsSubrequest, }; use quickwit_proto::indexing::ShardPositionsUpdate; +use quickwit_proto::ingest::ingester::IngesterStatus; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteShardsRequest, DeleteSourceRequest, EmptyResponse, FindIndexTemplateMatchesRequest, @@ -350,6 +349,28 @@ impl ControlPlane { } fn debug_info(&self) -> JsonValue { + // Build the union of ingesters tracked by ingester pool and the model. + let mut ingesters: BTreeMap = BTreeMap::new(); + + for (ingester_id, ingester) in self.ingest_controller.ingester_pool.keys_values() { + let ingester_json = json!({ + "available": true, + "status": ingester.status.as_json_str_name(), + }); + ingesters.insert(ingester_id.clone(), ingester_json); + } + for shard in self.model.all_shards() { + let ingester_id = NodeId::from(shard.leader_id.clone()); + + if let Entry::Vacant(entry) = ingesters.entry(ingester_id.clone()) { + let ingester_json = json!({ + "available": false, + "status": IngesterStatus::default(), + }); + entry.insert(ingester_json); + } + } + let physical_indexing_plan: Vec = self .indexing_scheduler .observable_state() @@ -392,6 +413,7 @@ impl ControlPlane { } } json!({ + "ingesters": ingesters, "physical_indexing_plan": physical_indexing_plan, "shard_table": per_index_and_leader_shards_json, }) @@ -1040,61 +1062,25 @@ fn apply_index_template_match( Ok(index_config) } -/// The indexer joined the cluster. #[derive(Debug)] -struct IndexerJoined(ClusterNode); +struct RebalanceShards; #[async_trait] -impl Handler for ControlPlane { +impl Handler for ControlPlane { type Reply = (); async fn handle( &mut self, - message: IndexerJoined, + _message: RebalanceShards, ctx: &ActorContext, ) -> Result { - info!( - "indexer `{}` joined the cluster: rebalancing shards and rebuilding indexing plan", - message.0.node_id() - ); - // TODO: Update shard table. - if let Err(metastore_error) = self + if let Err(error) = self .ingest_controller .rebalance_shards(&mut self.model, ctx.mailbox(), ctx.progress()) .await { - return convert_metastore_error::<()>(metastore_error).map(|_| ()); - } - self.indexing_scheduler.rebuild_plan(&self.model); - Ok(()) - } -} - -/// The indexer left the cluster. -#[derive(Debug)] -struct IndexerLeft(ClusterNode); - -#[async_trait] -impl Handler for ControlPlane { - type Reply = (); - - async fn handle( - &mut self, - message: IndexerLeft, - ctx: &ActorContext, - ) -> Result { - info!( - "indexer `{}` left the cluster: rebalancing shards and rebuilding indexing plan", - message.0.node_id() - ); - // TODO: Update shard table. - if let Err(metastore_error) = self - .ingest_controller - .rebalance_shards(&mut self.model, ctx.mailbox(), ctx.progress()) - .await - { - return convert_metastore_error::<()>(metastore_error).map(|_| ()); - } + return convert_metastore_error::<()>(error).map(|_| ()); + }; self.indexing_scheduler.rebuild_plan(&self.model); Ok(()) } @@ -1142,24 +1128,43 @@ async fn watcher_indexers( let Some(mailbox) = weak_mailbox.upgrade() else { return; }; + let mut trigger_rebalance = false; + match cluster_change { - ClusterChange::Add(node) => { - if node.enabled_services().contains(&QuickwitService::Indexer) - && let Err(error) = mailbox.send_message(IndexerJoined(node)).await - { - error!(%error, "failed to forward `IndexerJoined` event to control plane"); - } + ClusterChange::Add(node) if node.is_indexer() => { + info!( + "indexer `{}` joined the cluster: rebalancing shards and rebuilding indexing \ + plan", + node.node_id() + ); + trigger_rebalance = true; } - ClusterChange::Remove(node) => { - if node.enabled_services().contains(&QuickwitService::Indexer) - && let Err(error) = mailbox.send_message(IndexerLeft(node)).await - { - error!(%error, "failed to forward `IndexerLeft` event to control plane"); - } + ClusterChange::Remove(node) if node.is_indexer() => { + info!( + "indexer `{}` left the cluster: rebalancing shards and rebuilding indexing \ + plan", + node.node_id() + ); + trigger_rebalance = true } - ClusterChange::Update(_) => { - // We are not interested in updates (yet). + ClusterChange::Update { previous, updated } if updated.is_indexer() => { + let was_ready = previous.ingester_status().is_ready(); + let is_ready = updated.ingester_status().is_ready(); + + if was_ready ^ is_ready { + info!( + "indexer `{}` status changed to `{}`: rebalancing shards and rebuilding \ + indexing plan", + updated.node_id(), + updated.ingester_status().as_json_str_name() + ); + trigger_rebalance = true; + } } + _ => {} + } + if trigger_rebalance && mailbox.send_message(RebalanceShards).await.is_err() { + return; } } } @@ -1171,11 +1176,12 @@ mod tests { use mockall::Sequence; use quickwit_actors::{AskError, Observe, SupervisorMetrics}; - use quickwit_cluster::ClusterChangeStreamFactoryForTest; + use quickwit_cluster::{ClusterChangeStreamFactoryForTest, ClusterNode}; use quickwit_config::{ CLI_SOURCE_ID, INGEST_V2_SOURCE_ID, IndexConfig, KafkaSourceParams, SourceParams, }; use quickwit_indexing::IndexingService; + use quickwit_ingest::IngesterPoolEntry; use quickwit_metastore::{ CreateIndexRequestExt, IndexMetadata, ListIndexesMetadataResponseExt, }; @@ -1187,8 +1193,8 @@ mod tests { MockIndexingService, }; use quickwit_proto::ingest::ingester::{ - IngesterServiceClient, InitShardSuccess, InitShardsResponse, MockIngesterService, - RetainShardsResponse, + IngesterServiceClient, IngesterStatus, InitShardSuccess, InitShardsResponse, + MockIngesterService, RetainShardsResponse, }; use quickwit_proto::ingest::{Shard, ShardPKey, ShardState}; use quickwit_proto::metastore::{ @@ -2210,7 +2216,11 @@ mod tests { assert!(&retain_shards_for_source.shard_ids.is_empty()); Ok(RetainShardsResponse {}) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("node1".into(), ingester); let cluster_config = ClusterConfig::for_test(); @@ -2256,7 +2266,11 @@ mod tests { ); Ok(RetainShardsResponse {}) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("node1".into(), ingester); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); @@ -2448,7 +2462,7 @@ mod tests { let cluster_change = ClusterChange::Add(metastore_node); cluster_change_stream_tx.send(cluster_change).unwrap(); - let indexer_node = + let indexer_node: ClusterNode = ClusterNode::for_test("test-indexer", 1515, false, &["indexer"], &[]).await; let cluster_change = ClusterChange::Add(indexer_node.clone()); cluster_change_stream_tx.send(cluster_change).unwrap(); @@ -2456,11 +2470,35 @@ mod tests { let cluster_change = ClusterChange::Remove(indexer_node.clone()); cluster_change_stream_tx.send(cluster_change).unwrap(); - let IndexerJoined(joined) = control_plane_inbox.recv_typed_message().await.unwrap(); - assert_eq!(joined.grpc_advertise_addr().port(), 1516); + let RebalanceShards = control_plane_inbox.recv_typed_message().await.unwrap(); + let RebalanceShards = control_plane_inbox.recv_typed_message().await.unwrap(); + + // Test that a ClusterChange::Update with a status transition triggers rebalance. + let node_ready = ClusterNode::for_test_with_ingester_status( + "test-indexer", + 1515, + false, + &["indexer"], + &[], + IngesterStatus::Ready, + ) + .await; + let node_retiring = ClusterNode::for_test_with_ingester_status( + "test-indexer", + 1515, + false, + &["indexer"], + &[], + IngesterStatus::Retiring, + ) + .await; + let cluster_change = ClusterChange::Update { + previous: node_ready, + updated: node_retiring, + }; + cluster_change_stream_tx.send(cluster_change).unwrap(); - let IndexerLeft(left) = control_plane_inbox.recv_typed_message().await.unwrap(); - assert_eq!(left.grpc_advertise_addr().port(), 1516); + let RebalanceShards = control_plane_inbox.recv_typed_message().await.unwrap(); universe.assert_quit().await; } @@ -2493,7 +2531,7 @@ mod tests { disable_control_loop, ); let cluster_change_stream_tx = cluster_change_stream_factory.change_stream_tx(); - let indexer_node = + let indexer_node: ClusterNode = ClusterNode::for_test("test-indexer", 1515, false, &["indexer"], &[]).await; let cluster_change = ClusterChange::Add(indexer_node.clone()); cluster_change_stream_tx.send(cluster_change).unwrap(); @@ -2552,7 +2590,11 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id, ingester); let mut mock_metastore = MockMetastoreService::new(); @@ -2706,7 +2748,11 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id, ingester); let mut mock_metastore = MockMetastoreService::new(); diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 63295d61eca..89b669e4fa4 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::collections::btree_map::Entry; -use std::collections::{BTreeMap, BTreeSet, HashMap}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::fmt; use std::future::Future; use std::num::NonZeroUsize; @@ -34,7 +34,7 @@ use quickwit_proto::control_plane::{ GetOrCreateOpenShardsSuccess, }; use quickwit_proto::ingest::ingester::{ - CloseShardsRequest, CloseShardsResponse, IngesterService, InitShardFailure, + CloseShardsRequest, CloseShardsResponse, IngesterService, IngesterStatus, InitShardFailure, InitShardSubrequest, InitShardsRequest, InitShardsResponse, RetainShardsForSource, RetainShardsRequest, }; @@ -52,8 +52,7 @@ use rand::seq::SliceRandom; use rand::{Rng, RngCore, rng}; use serde::{Deserialize, Serialize}; use tokio::sync::{Mutex, OwnedMutexGuard}; -use tokio::task::JoinHandle; -use tracing::{Level, debug, enabled, error, info, warn}; +use tracing::{Level, debug, enabled, error, info, instrument, warn}; use ulid::Ulid; use super::scaling_arbiter::ScalingArbiter; @@ -202,12 +201,12 @@ pub struct IngestControllerStats { } pub struct IngestController { - ingester_pool: IngesterPool, + pub(crate) ingester_pool: IngesterPool, + pub(crate) stats: IngestControllerStats, metastore: MetastoreServiceClient, replication_factor: usize, // This lock ensures that only one rebalance operation is performed at a time. rebalance_lock: Arc>, - pub stats: IngestControllerStats, scaling_arbiter: ScalingArbiter, } @@ -350,8 +349,10 @@ impl IngestController { let operation: String = format!("retain shards `{ingester}`"); fire_and_forget( async move { - if let Err(retain_shards_err) = - ingester_client.retain_shards(retain_shards_req).await + if let Err(retain_shards_err) = ingester_client + .client + .retain_shards(retain_shards_req) + .await { error!(%retain_shards_err, "retain shards error"); } @@ -548,10 +549,12 @@ impl IngestController { // shards). let mut per_node_num_open_shards: HashMap = self .ingester_pool - .keys() + .keys_values() .into_iter() - .filter(|ingester| !unavailable_leaders.contains(ingester)) - .map(|ingester| (ingester, 0)) + .filter(|(ingester_id, ingester)| { + ingester.status.is_ready() && !unavailable_leaders.contains(ingester_id) + }) + .map(|(ingester_id, _)| (ingester_id, 0)) .collect(); let num_ingesters = per_node_num_open_shards.len(); @@ -648,7 +651,7 @@ impl IngestController { let init_shards_future = async move { let init_shards_result = tokio::time::timeout( INIT_SHARDS_REQUEST_TIMEOUT, - leader.init_shards(init_shards_request), + leader.client.init_shards(init_shards_request), ) .await; (leader_id.clone(), init_shards_result, init_shard_failures) @@ -910,7 +913,7 @@ impl IngestController { let close_shards_request = CloseShardsRequest { shard_pkeys }; if let Err(error) = progress - .protect_future(ingester.close_shards(close_shards_request)) + .protect_future(ingester.client.close_shards(close_shards_request)) .await { warn!("failed to scale down number of shards: {error}"); @@ -1008,15 +1011,16 @@ impl IngestController { /// /// This method is guarded by a lock to ensure that only one rebalance operation is performed at /// a time. + #[instrument(skip_all)] pub(crate) async fn rebalance_shards( &mut self, model: &mut ControlPlaneModel, mailbox: &Mailbox, progress: &Progress, - ) -> MetastoreResult>> { + ) -> MetastoreResult { let Ok(rebalance_guard) = self.rebalance_lock.clone().try_lock_owned() else { debug!("skipping rebalance: another rebalance is already in progress"); - return Ok(None); + return Ok(0); }; self.stats.num_rebalance_shards_ops += 1; @@ -1027,7 +1031,7 @@ impl IngestController { .set(shards_to_rebalance.len() as i64); if shards_to_rebalance.is_empty() { - return Ok(None); + return Ok(0); } let mut per_source_num_shards_to_open: HashMap = HashMap::new(); @@ -1036,7 +1040,6 @@ impl IngestController { .entry(shard.source_uid()) .or_default() += 1; } - let mut per_source_num_opened_shards: HashMap = self .try_open_shards( per_source_num_shards_to_open, @@ -1067,6 +1070,7 @@ impl IngestController { // Close as many shards as we opened. Because `try_open_shards` might fail partially, we // must only close the shards that we successfully opened. let mut shards_to_close = Vec::with_capacity(shards_to_rebalance.len()); + for shard in shards_to_rebalance { let source_uid = shard.source_uid(); let Some(num_open_shards) = per_source_num_opened_shards.get_mut(&source_uid) else { @@ -1078,10 +1082,8 @@ impl IngestController { *num_open_shards -= 1; shards_to_close.push(shard); } - - let mailbox_clone = mailbox.clone(); - let close_shards_fut = self.close_shards(shards_to_close); + let mailbox_clone = mailbox.clone(); let close_shards_and_send_callback_fut = async move { // We wait for a few seconds before closing the shards to give the ingesters some time @@ -1099,8 +1101,12 @@ impl IngestController { }; let _ = mailbox_clone.send_message(callback).await; }; + tokio::spawn(close_shards_and_send_callback_fut); - Ok(Some(tokio::spawn(close_shards_and_send_callback_fut))) + if num_opened_shards > 0 { + info!("rebalance opened {num_opened_shards} new shards"); + } + Ok(num_opened_shards) } /// Computes shards that need to be rebalanced. @@ -1114,31 +1120,41 @@ impl IngestController { /// The closing operation can only be done by the leader of that shard. /// For these reason, we exclude these shards from the rebalance process. fn compute_shards_to_rebalance(&self, model: &ControlPlaneModel) -> Vec { - let mut per_available_ingester_shards: HashMap> = self - .ingester_pool - .keys() - .into_iter() - .map(|ingester_id| (ingester_id, Vec::new())) - .collect(); + let mut per_ready_ingester_shards: HashMap> = HashMap::new(); + let mut retiring_ingesters: HashSet = HashSet::new(); + + for (ingester_id, ingester) in self.ingester_pool.keys_values() { + if ingester.status.is_ready() { + per_ready_ingester_shards.insert(ingester_id, Vec::new()); + } else if ingester.status == IngesterStatus::Retiring { + retiring_ingesters.insert(ingester_id); + } + } + + let mut shards_to_rebalance: Vec = Vec::new(); + let mut num_ready_shards: usize = 0; - let mut num_available_shards: usize = 0; for shard in model.all_shards() { if !shard.is_open() { continue; } let leader_id_ref = NodeIdRef::from_str(&shard.leader_id); - if let Some(shards) = per_available_ingester_shards.get_mut(leader_id_ref) { - // We only consider shards that are on available ingesters - // because we won't be able to move shards that are not reachable. - num_available_shards += 1; - shards.push(&shard.shard) + + if let Some(shards) = per_ready_ingester_shards.get_mut(leader_id_ref) { + // Shards on ready ingesters participate in the balancing logic. + num_ready_shards += 1; + shards.push(&shard.shard); + } else if retiring_ingesters.contains(leader_id_ref) { + // All open shards on retiring ingesters must be rebalanced. + shards_to_rebalance.push(shard.shard.clone()); } } - let num_available_ingesters = per_available_ingester_shards.len(); + let num_retiring_shards = shards_to_rebalance.len(); + let num_ready_ingesters = per_ready_ingester_shards.len(); let mut rng = rng(); - let mut per_leader_open_shards_shuffled: Vec> = per_available_ingester_shards + let mut per_leader_open_shards_shuffled: Vec> = per_ready_ingester_shards .into_values() .map(|mut shards| { shards.shuffle(&mut rng); @@ -1146,11 +1162,9 @@ impl IngestController { }) .collect(); - let mut shards_to_rebalance: Vec = Vec::new(); - // This is more of a loop-loop, but since we know it should exit before - // `num_available_ingesters`, we defensively use a for-loop. - for _ in 0..num_available_shards { + // `num_ready_shards`, we defensively use a for-loop. + for _ in 0..num_ready_shards { let MinMaxResult::MinMax(min_shards, max_shards) = per_leader_open_shards_shuffled .iter_mut() .minmax_by_key(|shards| shards.len()) @@ -1177,13 +1191,13 @@ impl IngestController { debug!("no shards to rebalance"); } else { info!( - num_available_shards, - num_available_ingesters, + num_ready_shards, + num_ready_ingesters, + num_retiring_shards, num_shards_to_rebalance = shards_to_rebalance.len(), "rebalancing shards" ); } - shards_to_rebalance } @@ -1219,7 +1233,7 @@ impl IngestController { let close_shards_future = async move { tokio::time::timeout( CLOSE_SHARDS_REQUEST_TIMEOUT, - ingester.close_shards(shards_to_close_request), + ingester.client.close_shards(shards_to_close_request), ) .await }; @@ -1313,12 +1327,12 @@ mod tests { use quickwit_common::shared_consts::DEFAULT_SHARD_THROUGHPUT_LIMIT; use quickwit_common::tower::DelayLayer; use quickwit_config::{DocMapping, INGEST_V2_SOURCE_ID, SourceConfig}; - use quickwit_ingest::{RateMibPerSec, ShardInfo}; + use quickwit_ingest::{IngesterPoolEntry, RateMibPerSec, ShardInfo}; use quickwit_metastore::IndexMetadata; use quickwit_proto::control_plane::GetOrCreateOpenShardsSubrequest; use quickwit_proto::ingest::ingester::{ - CloseShardsResponse, IngesterServiceClient, InitShardSuccess, InitShardsResponse, - MockIngesterService, RetainShardsResponse, + CloseShardsResponse, IngesterServiceClient, IngesterStatus, InitShardSuccess, + InitShardsResponse, MockIngesterService, RetainShardsResponse, }; use quickwit_proto::ingest::{IngestV2Error, Shard, ShardState}; use quickwit_proto::metastore::{ @@ -1382,10 +1396,14 @@ mod tests { let metastore = MetastoreServiceClient::from_mock(mock_metastore); let mock_ingester = MockIngesterService::new(); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - ingester_pool.insert(NodeId::from("test-ingester-1"), ingester.clone()); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); let mut mock_ingester = MockIngesterService::new(); let index_uid_1_clone = index_uid_1.clone(); @@ -1412,8 +1430,12 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); - ingester_pool.insert(NodeId::from("test-ingester-2"), ingester.clone()); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from("test-ingester-2"), ingester); let replication_factor = 2; let mut controller = IngestController::new( @@ -1596,10 +1618,14 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - ingester_pool.insert(NodeId::from("test-ingester-1"), ingester.clone()); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); let replication_factor = 1; let mut controller = IngestController::new( @@ -1710,10 +1736,12 @@ mod tests { controller.allocate_shards(0, &FnvHashSet::default(), &model); assert!(leader_follower_pairs_opt.is_none()); - ingester_pool.insert( - NodeId::from("test-ingester-1"), - IngesterServiceClient::mocked(), - ); + let client_1 = IngesterServiceClient::mocked(); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester_1); let leader_follower_pairs_opt = controller.allocate_shards(0, &FnvHashSet::default(), &model); @@ -1722,7 +1750,12 @@ mod tests { // find any solution. assert!(leader_follower_pairs_opt.is_none()); - ingester_pool.insert("test-ingester-2".into(), IngesterServiceClient::mocked()); + let client_2 = IngesterServiceClient::mocked(); + let ingester_2 = IngesterPoolEntry { + client: client_2, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-2".into(), ingester_2); let leader_follower_pairs = controller .allocate_shards(0, &FnvHashSet::default(), &model) @@ -1841,7 +1874,12 @@ mod tests { Some(NodeId::from("test-ingester-1")) ); - ingester_pool.insert("test-ingester-3".into(), IngesterServiceClient::mocked()); + let client = IngesterServiceClient::mocked(); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-3".into(), ingester); let unavailable_leaders = FnvHashSet::from_iter([NodeId::from("test-ingester-2")]); let leader_follower_pairs = controller .allocate_shards(4, &unavailable_leaders, &model) @@ -1933,7 +1971,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_0, ingester_0); let ingester_id_1 = NodeId::from("test-ingester-1"); @@ -1955,16 +1997,24 @@ mod tests { Err(IngestV2Error::Internal("internal error".to_string())) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_1, ingester_1); let ingester_id_2 = NodeId::from("test-ingester-2"); let mut mock_ingester_2 = MockIngesterService::new(); mock_ingester_2.expect_init_shards().never(); - let ingester_2 = IngesterServiceClient::tower() + let client_2 = IngesterServiceClient::tower() .stack_init_shards_layer(DelayLayer::new(INIT_SHARDS_REQUEST_TIMEOUT * 2)) .build_from_mock(mock_ingester_2); + let ingester_2 = IngesterPoolEntry { + client: client_2, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_2, ingester_2); let init_shards_response = controller @@ -2171,10 +2221,12 @@ mod tests { Ok(response) }); - ingester_pool.insert( - NodeId::from("test-ingester-1"), - IngesterServiceClient::from_mock(mock_ingester), - ); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); let source_uids: HashMap = HashMap::from_iter([(source_uid.clone(), 1)]); let unavailable_leaders = FnvHashSet::default(); let progress = Progress::default(); @@ -2343,7 +2395,11 @@ mod tests { "failed to close shards".to_string(), )) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester".into(), ingester); let shard_infos = BTreeSet::from_iter([ @@ -2496,7 +2552,11 @@ mod tests { }, ); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester".into(), ingester); let shard_infos = BTreeSet::from_iter([ShardInfo { @@ -2642,7 +2702,11 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester".into(), ingester); // Test failed to open shards. @@ -2764,7 +2828,11 @@ mod tests { }; Ok(response) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester".into(), ingester); // Test failed to close shard. @@ -2994,18 +3062,24 @@ mod tests { count_calls_clone.fetch_add(1, Ordering::Release); Ok(RetainShardsResponse {}) }); - ingester_pool.insert( - "node-1".into(), - IngesterServiceClient::from_mock(mock_ingester_1), - ); - ingester_pool.insert( - "node-2".into(), - IngesterServiceClient::from_mock(mock_ingester_2), - ); - ingester_pool.insert( - "node-3".into(), - IngesterServiceClient::from_mock(mock_ingester_3), - ); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("node-1".into(), ingester_1); + let client_2 = IngesterServiceClient::from_mock(mock_ingester_2); + let ingester_2 = IngesterPoolEntry { + client: client_2, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("node-2".into(), ingester_2); + let client_3 = IngesterServiceClient::from_mock(mock_ingester_3); + let ingester_3 = IngesterPoolEntry { + client: client_3, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("node-3".into(), ingester_3); let node_id = "node-1".into(); let wait_handle = controller.sync_with_ingester(&node_id, &model); wait_handle.wait().await; @@ -3133,7 +3207,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_0.clone(), ingester_0); let ingester_id_1 = NodeId::from("test-ingester-1"); @@ -3151,16 +3229,24 @@ mod tests { Err(IngestV2Error::Internal("internal error".to_string())) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_1.clone(), ingester_1); let ingester_id_2 = NodeId::from("test-ingester-2"); let mut mock_ingester_2 = MockIngesterService::new(); mock_ingester_2.expect_close_shards().never(); - let ingester_2 = IngesterServiceClient::tower() + let client_2 = IngesterServiceClient::tower() .stack_close_shards_layer(DelayLayer::new(CLOSE_SHARDS_REQUEST_TIMEOUT * 2)) .build_from_mock(mock_ingester_2); + let ingester_2 = IngesterPoolEntry { + client: client_2, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_2.clone(), ingester_2); // In this test: @@ -3261,11 +3347,11 @@ mod tests { let (control_plane_mailbox, control_plane_inbox) = universe.create_test_mailbox(); let progress = Progress::default(); - let close_shards_task_opt = controller + let num_opened_shards = controller .rebalance_shards(&mut model, &control_plane_mailbox, &progress) .await .unwrap(); - assert!(close_shards_task_opt.is_none()); + assert_eq!(num_opened_shards, 0); let index_metadata = IndexMetadata::for_test("test-index", "ram://indexes/test-index"); let index_uid = index_metadata.index_uid.clone(); @@ -3340,7 +3426,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_0.clone(), ingester_0); let ingester_id_1 = NodeId::from("test-ingester-1"); @@ -3382,24 +3472,26 @@ mod tests { }; Ok(response) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert(ingester_id_1.clone(), ingester_1); - let close_shards_task = controller + let num_opened_shards = controller .rebalance_shards(&mut model, &control_plane_mailbox, &progress) .await - .unwrap() .unwrap(); + assert_eq!(num_opened_shards, 1); - tokio::time::timeout(CLOSE_SHARDS_REQUEST_TIMEOUT * 2, close_shards_task) - .await - .unwrap() - .unwrap(); - - let callbacks: Vec = control_plane_inbox.drain_for_test_typed(); - assert_eq!(callbacks.len(), 1); - - let callback = &callbacks[0]; + let callback: RebalanceShardsCallback = tokio::time::timeout( + CLOSE_SHARDS_REQUEST_TIMEOUT * 2, + control_plane_inbox.recv_typed_message(), + ) + .await + .unwrap() + .unwrap(); assert_eq!(callback.closed_shards.len(), 1); } @@ -3544,8 +3636,9 @@ mod tests { /// - `available_ingester_shards`: open shards per available ingester /// - `unavailable_ingester_shards`: open shards on unavailable ingesters fn test_compute_shards_to_rebalance_aux( - available_ingester_shards: &[usize], + ready_ingester_shards: &[usize], unavailable_ingester_shards: &[usize], + retiring_ingester_shards: &[usize], ) { let index_id = "test-index"; let index_metadata = IndexMetadata::for_test(index_id, "ram://indexes/test-index"); @@ -3563,28 +3656,44 @@ mod tests { let mock_ingester = MockIngesterService::new(); let ingester_client = IngesterServiceClient::from_mock(mock_ingester); - let active_ids: Vec = (0..available_ingester_shards.len()) - .map(|i| format!("active-ingester-{}", i)) + let ready_ids: Vec = (0..ready_ingester_shards.len()) + .map(|i| format!("ready-ingester-{}", i)) .collect(); - for ingester_id in &active_ids { - ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester_client.clone()); + for ingester_id in &ready_ids { + let ingester = IngesterPoolEntry { + client: ingester_client.clone(), + status: IngesterStatus::Ready, + }; + ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); } - let inactive_ids: Vec = (0..unavailable_ingester_shards.len()) - .map(|i| format!("inactive-ingester-{}", i)) + let unavailable_ids: Vec = (0..unavailable_ingester_shards.len()) + .map(|i| format!("unavailable-ingester-{}", i)) + .collect(); + + let retiring_ids: Vec = (0..retiring_ingester_shards.len()) + .map(|i| format!("retiring-ingester-{}", i)) .collect(); + for ingester_id in &retiring_ids { + let ingester = IngesterPoolEntry { + client: ingester_client.clone(), + status: IngesterStatus::Retiring, + }; + ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); + } + let mut shards: Vec = Vec::new(); let mut shard_id: u64 = 0; - for (idx, &num_shards) in available_ingester_shards.iter().enumerate() { + for (idx, &num_shards) in ready_ingester_shards.iter().enumerate() { for _ in 0..num_shards { shards.push(Shard { index_uid: Some(index_uid.clone()), source_id: source_id.clone(), shard_id: Some(ShardId::from(shard_id)), - leader_id: active_ids[idx].clone(), + leader_id: ready_ids[idx].clone(), shard_state: ShardState::Open as i32, ..Default::default() }); @@ -3599,7 +3708,24 @@ mod tests { index_uid: Some(index_uid.clone()), source_id: source_id.clone(), shard_id: Some(ShardId::from(shard_id)), - leader_id: inactive_ids[idx].clone(), + leader_id: unavailable_ids[idx].clone(), + shard_state: ShardState::Open as i32, + ..Default::default() + }); + shard_id += 1; + } + } + + let num_retiring_shards: usize = retiring_ingester_shards.iter().sum(); + + // Shards on retiring ingesters - all of these should be rebalanced + for (idx, &num_shards) in retiring_ingester_shards.iter().enumerate() { + for _ in 0..num_shards { + shards.push(Shard { + index_uid: Some(index_uid.clone()), + source_id: source_id.clone(), + shard_id: Some(ShardId::from(shard_id)), + leader_id: retiring_ids[idx].clone(), shard_state: ShardState::Open as i32, ..Default::default() }); @@ -3618,6 +3744,13 @@ mod tests { ); let shards_to_rebalance = controller.compute_shards_to_rebalance(&model); + // All shards on retiring ingesters must be rebalanced. + let num_retiring_shards_to_rebalance = shards_to_rebalance + .iter() + .filter(|shard| shard.leader_id.starts_with("retiring-")) + .count(); + assert_eq!(num_retiring_shards_to_rebalance, num_retiring_shards); + let source_uid = SourceUid { index_uid: index_uid.clone(), source_id: source_id.clone(), @@ -3630,9 +3763,9 @@ mod tests { let closed_shard_ids = model.close_shards(&source_uid, &shard_ids_to_rebalance); assert_eq!(closed_shard_ids.len(), shards_to_rebalance.len()); - let mut per_available_ingester_num_shards: HashMap<&str, usize> = active_ids + let mut per_ready_ingester_num_shards: HashMap<&str, usize> = ready_ids .iter() - .map(|active_id| (active_id.as_str(), 0)) + .map(|ready_id| (ready_id.as_str(), 0)) .collect(); for shard in model.all_shards() { @@ -3640,68 +3773,85 @@ mod tests { continue; } if let Some(count_shard) = - per_available_ingester_num_shards.get_mut(shard.leader_id.as_str()) + per_ready_ingester_num_shards.get_mut(shard.leader_id.as_str()) { *count_shard += 1; } } - // Now we move the different shards. - let mut per_ingester_num_shards_sorted: BTreeSet<(usize, &str)> = - per_available_ingester_num_shards - .into_iter() - .map(|(ingester_id, num_shards)| (num_shards, ingester_id)) - .collect(); - let mut opened_shards: Vec = Vec::new(); - for _ in 0..shards_to_rebalance.len() { - let (num_shards, ingester_id) = per_ingester_num_shards_sorted.pop_first().unwrap(); - let opened_shard = Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.to_string(), - shard_id: Some(ShardId::from(shard_id)), - leader_id: ingester_id.to_string(), - shard_state: ShardState::Open as i32, - ..Default::default() - }; - per_ingester_num_shards_sorted.insert((num_shards + 1, ingester_id)); - opened_shards.push(opened_shard); - shard_id += 1; - } + // Now we move the different shards to ready ingesters (not retiring ones). + // We can only simulate this if there are ready ingesters to receive shards. + if !ready_ids.is_empty() { + let mut per_ingester_num_shards_sorted: BTreeSet<(usize, &str)> = + per_ready_ingester_num_shards + .into_iter() + .map(|(ingester_id, num_shards)| (num_shards, ingester_id)) + .collect(); + let mut opened_shards: Vec = Vec::new(); + for _ in 0..shards_to_rebalance.len() { + let (num_shards, ingester_id) = per_ingester_num_shards_sorted.pop_first().unwrap(); + let opened_shard = Shard { + index_uid: Some(index_uid.clone()), + source_id: source_id.to_string(), + shard_id: Some(ShardId::from(shard_id)), + leader_id: ingester_id.to_string(), + shard_state: ShardState::Open as i32, + ..Default::default() + }; + per_ingester_num_shards_sorted.insert((num_shards + 1, ingester_id)); + opened_shards.push(opened_shard); + shard_id += 1; + } - if let Some((min_shards, max_shards)) = per_ingester_num_shards_sorted - .iter() - .map(|(num_shards, _)| num_shards) - .copied() - .minmax() - .into_option() - { - assert!(min_shards + min_shards.div_ceil(10).max(2) >= max_shards); - } + if let Some((min_shards, max_shards)) = per_ingester_num_shards_sorted + .iter() + .map(|(num_shards, _)| num_shards) + .copied() + .minmax() + .into_option() + { + assert!(min_shards + min_shards.div_ceil(10).max(2) >= max_shards); + } - // Test stability of the algorithm - model.insert_shards(&index_uid, &source_id, opened_shards); + // Test stability of the algorithm: mark the retiring ingesters as + // decommissioned, insert the new shards, and verify no further rebalance is + // needed among the ready ingesters. + for ingester_id in &retiring_ids { + let ingester = IngesterPoolEntry { + client: ingester_client.clone(), + status: IngesterStatus::Decommissioned, + }; + ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); + } + model.insert_shards(&index_uid, &source_id, opened_shards); - let shards_to_rebalance = controller.compute_shards_to_rebalance(&model); - assert!(shards_to_rebalance.is_empty()); + let shards_to_rebalance = controller.compute_shards_to_rebalance(&model); + assert!(shards_to_rebalance.is_empty()); + } } proptest! { #[test] fn test_compute_shards_to_rebalance_proptest( - active_shards in proptest::collection::vec(0..13usize, 0..13usize), - inactive_shards in proptest::collection::vec(0..13usize, 0..5usize), + ready_shards in proptest::collection::vec(0..13usize, 0..13usize), + unavailable_shards in proptest::collection::vec(0..13usize, 0..5usize), + retiring_shards in proptest::collection::vec(0..5usize, 0..5usize), ) { - test_compute_shards_to_rebalance_aux(&active_shards, &inactive_shards); + test_compute_shards_to_rebalance_aux(&ready_shards, &unavailable_shards, &retiring_shards); } } #[test] fn test_compute_shards_to_rebalance() { - test_compute_shards_to_rebalance_aux(&[], &[]); - test_compute_shards_to_rebalance_aux(&[0], &[]); - test_compute_shards_to_rebalance_aux(&[1], &[]); - test_compute_shards_to_rebalance_aux(&[0, 1], &[]); - test_compute_shards_to_rebalance_aux(&[0, 1], &[1]); - test_compute_shards_to_rebalance_aux(&[0, 1, 2], &[3, 4]); + test_compute_shards_to_rebalance_aux(&[], &[], &[]); + test_compute_shards_to_rebalance_aux(&[0], &[], &[]); + test_compute_shards_to_rebalance_aux(&[1], &[], &[]); + test_compute_shards_to_rebalance_aux(&[0, 1], &[], &[]); + test_compute_shards_to_rebalance_aux(&[0, 1], &[1], &[]); + test_compute_shards_to_rebalance_aux(&[0, 1, 2], &[3, 4], &[]); + // Retiring ingesters: all their shards must be rebalanced + test_compute_shards_to_rebalance_aux(&[1, 1], &[], &[3]); + test_compute_shards_to_rebalance_aux(&[0, 0, 0], &[], &[5]); + test_compute_shards_to_rebalance_aux(&[2], &[], &[1, 2]); } } diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 63c746aabe0..273f2df5b37 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -361,6 +361,7 @@ impl IngestSource { }; for num_attempts in 1..=retry_params.max_attempts { let Err(error) = ingester + .client .truncate_shards(truncate_shards_request.clone()) .await else { @@ -672,9 +673,11 @@ mod tests { use quickwit_common::metrics::MEMORY_METRICS; use quickwit_common::stream_utils::InFlightValue; use quickwit_config::{IndexingSettings, SourceConfig, SourceParams}; + use quickwit_ingest::IngesterPoolEntry; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::ingest::ingester::{ - FetchMessage, IngesterServiceClient, MockIngesterService, TruncateShardsResponse, + FetchMessage, IngesterServiceClient, IngesterStatus, MockIngesterService, + TruncateShardsResponse, }; use quickwit_proto::ingest::{IngestV2Error, MRecordBatch, Shard, ShardState}; use quickwit_proto::metastore::{AcquireShardsResponse, MockMetastoreService}; @@ -929,8 +932,12 @@ mod tests { Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); - ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); let event_broker = EventBroker::default(); @@ -1126,8 +1133,12 @@ mod tests { Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); - ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = @@ -1291,8 +1302,12 @@ mod tests { Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); - ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = @@ -1599,8 +1614,12 @@ mod tests { }) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); - ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); let event_broker = EventBroker::default(); let source_runtime = SourceRuntime { @@ -1699,8 +1718,12 @@ mod tests { Ok(TruncateShardsResponse {}) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); - ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); let mut mock_ingester_1 = MockIngesterService::new(); mock_ingester_1 @@ -1726,8 +1749,12 @@ mod tests { Ok(TruncateShardsResponse {}) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); - ingester_pool.insert("test-ingester-1".into(), ingester_1.clone()); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-1".into(), ingester_1); let mut mock_ingester_3 = MockIngesterService::new(); mock_ingester_3 @@ -1746,8 +1773,12 @@ mod tests { Ok(TruncateShardsResponse {}) }); - let ingester_3 = IngesterServiceClient::from_mock(mock_ingester_3); - ingester_pool.insert("test-ingester-3".into(), ingester_3.clone()); + let client_3 = IngesterServiceClient::from_mock(mock_ingester_3); + let ingester_3 = IngesterPoolEntry { + client: client_3, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-3".into(), ingester_3); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs index 9bbbe94bb47..98be3df36a1 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs @@ -562,7 +562,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let (_temp_dir, state) = IngesterState::for_test().await; + let (_temp_dir, state) = IngesterState::for_test(cluster.clone()).await; let weak_state = state.weak(); let mut task = BroadcastLocalShardsTask { cluster, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index 6e8d085e35d..e20ad123a2e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -495,7 +495,11 @@ async fn fault_tolerant_fetch_stream( shard_id: Some(shard_id.clone()), from_position_exclusive: Some(from_position_exclusive.clone()), }; - let mut fetch_stream = match ingester.open_fetch_stream(open_fetch_stream_request).await { + let mut fetch_stream = match ingester + .client + .open_fetch_stream(open_fetch_stream_request) + .await + { Ok(fetch_stream) => fetch_stream, Err(not_found_error @ IngestV2Error::ShardNotFound { .. }) => { error!( @@ -623,12 +627,15 @@ pub(super) mod tests { use bytes::Bytes; use quickwit_proto::ingest::ShardState; - use quickwit_proto::ingest::ingester::{IngesterServiceClient, MockIngesterService}; + use quickwit_proto::ingest::ingester::{ + IngesterServiceClient, IngesterStatus, MockIngesterService, + }; use quickwit_proto::types::queue_id; use tokio::time::timeout; use super::*; use crate::MRecord; + use crate::ingest_v2::IngesterPoolEntry; pub fn into_fetch_payload(fetch_message: FetchMessage) -> FetchPayload { match fetch_message.message.unwrap() { @@ -1325,8 +1332,11 @@ pub(super) mod tests { Ok(service_stream_1) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); - + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-1".into(), ingester_1); let fetch_payload = FetchPayload { @@ -1425,7 +1435,11 @@ pub(super) mod tests { "open fetch stream error".to_string(), )) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; let mut mock_ingester_1 = MockIngesterService::new(); let index_uid_clone = index_uid.clone(); @@ -1440,7 +1454,11 @@ pub(super) mod tests { Ok(service_stream_1) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); @@ -1540,7 +1558,11 @@ pub(super) mod tests { Ok(service_stream_0) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; let mut mock_ingester_1 = MockIngesterService::new(); let index_uid_clone = index_uid.clone(); @@ -1555,7 +1577,11 @@ pub(super) mod tests { Ok(service_stream_1) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); @@ -1658,7 +1684,11 @@ pub(super) mod tests { shard_id: ShardId::from(1), }) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0); fault_tolerant_fetch_stream( @@ -1746,8 +1776,11 @@ pub(super) mod tests { Ok(service_stream_2) }); - let ingester = IngesterServiceClient::from_mock(mock_ingester); - + let client = IngesterServiceClient::from_mock(mock_ingester); + let ingester = IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester".into(), ingester); let fetch_payload = FetchPayload { diff --git a/quickwit/quickwit-ingest/src/ingest_v2/helpers.rs b/quickwit/quickwit-ingest/src/ingest_v2/helpers.rs new file mode 100644 index 00000000000..5422d6c7ebc --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/helpers.rs @@ -0,0 +1,273 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::time::{Duration, Instant}; + +use anyhow::{Context, anyhow, bail}; +use futures::StreamExt; +use quickwit_common::pretty::PrettyDisplay; +use quickwit_proto::ingest::ingester::{ + DecommissionRequest, IngesterService, IngesterStatus, OpenObservationStreamRequest, +}; +use tracing::info; + +/// Tries to get the current status of an ingester by opening an observation stream +/// and reading the first message. +/// +/// # Errors +/// +/// Returns an error if: +/// - The observation stream fails to open +/// - The stream ends without producing a message +/// - The stream ends after returning an error +pub async fn try_get_ingester_status( + ingester: &impl IngesterService, +) -> anyhow::Result { + let mut observation_stream = ingester + .open_observation_stream(OpenObservationStreamRequest {}) + .await + .context("failed to open observation stream")?; + + let next_observation_message = observation_stream + .next() + .await + .context("observation stream ended")? + .context("observation stream failed")?; + + Ok(next_observation_message.status()) +} + +/// Waits for an ingester to reach a specific status by monitoring its observation stream. +/// +/// This function continuously polls the observation stream until the ingester reaches +/// the desired status. +/// +/// # Errors +/// +/// Returns an error if: +/// - The observation stream fails to open +/// - The stream ends without producing a message +/// - The stream ends after returning an error +/// - The timeout is exceeded +pub async fn wait_for_ingester_status( + ingester: &impl IngesterService, + status: IngesterStatus, + timeout_after: Duration, +) -> anyhow::Result<()> { + debug_assert!( + timeout_after > Duration::ZERO, + "timeout_after should be greater than zero" + ); + tokio::time::timeout( + timeout_after, + wait_for_ingester_status_inner(ingester, status), + ) + .await + .with_context(|| { + format!( + "timed out waiting for ingester to transition to status {status} after {} seconds", + timeout_after.as_secs(), + ) + })? +} + +async fn wait_for_ingester_status_inner( + ingester: &impl IngesterService, + status: IngesterStatus, +) -> anyhow::Result<()> { + let mut observation_stream = ingester + .open_observation_stream(OpenObservationStreamRequest {}) + .await + .context("failed to open observation stream")?; + + loop { + match observation_stream.next().await { + Some(Ok(observation_message)) => { + if observation_message.status() == status { + return Ok(()); + } + } + Some(Err(error)) => { + return Err(anyhow!(error).context("observation stream failed")); + } + None => { + bail!("observation stream ended"); + } + } + } +} + +/// Initiates decommission of an ingester and waits for it to complete. +/// +/// This function sends a decommission request to the ingester and then waits +/// for it to reach the `Decommissioned` status. +/// +/// # Errors +/// +/// Returns an error if: +/// - The decommission request fails +/// - The observation stream fails to open +/// - The stream ends without producing a message +/// - The stream ends after returning an error +/// - The timeout is exceeded +pub async fn wait_for_ingester_decommission( + ingester: &impl IngesterService, + timeout_after: Duration, +) -> anyhow::Result<()> { + let now = Instant::now(); + + ingester + .decommission(DecommissionRequest {}) + .await + .context("failed to initiate ingester decommission")?; + + wait_for_ingester_status( + ingester, + IngesterStatus::Decommissioned, + timeout_after.saturating_sub(now.elapsed()), + ) + .await?; + + info!( + "successfully decommissioned ingester in {}", + now.elapsed().pretty_display() + ); + Ok(()) +} + +#[cfg(test)] +mod tests { + + use std::time::Duration; + + use quickwit_common::ServiceStream; + use quickwit_proto::ingest::ingester::{ + DecommissionResponse, IngesterServiceClient, MockIngesterService, ObservationMessage, + }; + + use super::*; + + #[tokio::test] + async fn test_try_get_ingester_status() { + let mut mock_ingester = MockIngesterService::new(); + mock_ingester + .expect_open_observation_stream() + .once() + .returning(|_| { + let (service_stream_tx, service_stream) = ServiceStream::new_bounded(1); + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Initializing as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + Ok(service_stream) + }); + let ingester = IngesterServiceClient::from_mock(mock_ingester); + let status = try_get_ingester_status(&ingester).await.unwrap(); + assert_eq!(status, IngesterStatus::Initializing); + } + + #[tokio::test] + async fn test_wait_for_ingester_status() { + let mut mock_ingester = MockIngesterService::new(); + mock_ingester + .expect_open_observation_stream() + .once() + .returning(|_| { + let (service_stream_tx, service_stream) = ServiceStream::new_bounded(2); + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Initializing as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Ready as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + Ok(service_stream) + }); + let ingester = IngesterServiceClient::from_mock(mock_ingester); + wait_for_ingester_status(&ingester, IngesterStatus::Ready, Duration::from_secs(1)) + .await + .unwrap(); + } + + #[tokio::test] + async fn test_wait_for_ingester_decommission_elapsed_timeout_not_zero() { + let mut mock_ingester = MockIngesterService::new(); + mock_ingester + .expect_open_observation_stream() + .once() + .returning(|_| { + let (service_stream_tx, service_stream) = ServiceStream::new_bounded(1); + // Simulate the ingester transitioning to Decommissioned after 50ms. + tokio::spawn(async move { + tokio::time::sleep(Duration::from_millis(50)).await; + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Decommissioned as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + }); + Ok(service_stream) + }); + mock_ingester + .expect_decommission() + .once() + .returning(|_| Ok(DecommissionResponse {})); + let ingester = IngesterServiceClient::from_mock(mock_ingester); + wait_for_ingester_decommission(&ingester, Duration::from_secs(1)) + .await + .unwrap(); + } + + #[tokio::test] + async fn test_wait_for_ingester_decommission() { + let mut mock_ingester = MockIngesterService::new(); + mock_ingester + .expect_open_observation_stream() + .once() + .returning(|_| { + let (service_stream_tx, service_stream) = ServiceStream::new_bounded(3); + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Ready as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Decommissioning as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + + let message = ObservationMessage { + node_id: "test-ingester".to_string(), + status: IngesterStatus::Decommissioned as i32, + }; + service_stream_tx.try_send(Ok(message)).unwrap(); + Ok(service_stream) + }); + mock_ingester + .expect_decommission() + .once() + .returning(|_| Ok(DecommissionResponse {})); + let ingester = IngesterServiceClient::from_mock(mock_ingester); + wait_for_ingester_decommission(&ingester, Duration::from_secs(1)) + .await + .unwrap(); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs index 14f535be4f2..5b91dc8025e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/idle.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/idle.rs @@ -78,7 +78,8 @@ impl CloseIdleShardsTask { #[cfg(test)] mod tests { - + use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; + use quickwit_config::service::QuickwitService; use quickwit_proto::types::{IndexUid, ShardId}; use super::*; @@ -87,7 +88,15 @@ mod tests { #[tokio::test] async fn test_close_idle_shards_run() { - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let weak_state = state.weak(); let idle_shard_timeout = RUN_INTERVAL_PERIOD * 4; let join_handle = CloseIdleShardsTask::spawn(weak_state, idle_shard_timeout); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 65c268881ac..3258a528fbd 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -19,7 +19,6 @@ use std::path::Path; use std::sync::Arc; use std::time::{Duration, Instant}; -use anyhow::Context; use async_trait::async_trait; use bytesize::ByteSize; use futures::StreamExt; @@ -31,7 +30,6 @@ use quickwit_common::metrics::{GaugeGuard, MEMORY_METRICS}; use quickwit_common::pretty::PrettyDisplay; use quickwit_common::pubsub::{EventBroker, EventSubscriber}; use quickwit_common::rate_limiter::{RateLimiter, RateLimiterSettings}; -use quickwit_common::tower::Pool; use quickwit_common::{ServiceStream, rate_limited_error, rate_limited_warn}; use quickwit_proto::control_plane::{ AdviseResetShardsRequest, ControlPlaneService, ControlPlaneServiceClient, @@ -39,13 +37,13 @@ use quickwit_proto::control_plane::{ use quickwit_proto::indexing::ShardPositionsUpdate; use quickwit_proto::ingest::ingester::{ AckReplicationMessage, CloseShardsRequest, CloseShardsResponse, DecommissionRequest, - DecommissionResponse, FetchMessage, IngesterService, IngesterServiceClient, - IngesterServiceStream, IngesterStatus, InitShardFailure, InitShardSuccess, InitShardsRequest, - InitShardsResponse, ObservationMessage, OpenFetchStreamRequest, OpenObservationStreamRequest, - OpenReplicationStreamRequest, OpenReplicationStreamResponse, PersistFailure, - PersistFailureReason, PersistRequest, PersistResponse, PersistSuccess, ReplicateFailureReason, - ReplicateSubrequest, RetainShardsForSource, RetainShardsRequest, RetainShardsResponse, - SynReplicationMessage, TruncateShardsRequest, TruncateShardsResponse, + DecommissionResponse, FetchMessage, IngesterService, IngesterServiceStream, IngesterStatus, + InitShardFailure, InitShardSuccess, InitShardsRequest, InitShardsResponse, ObservationMessage, + OpenFetchStreamRequest, OpenObservationStreamRequest, OpenReplicationStreamRequest, + OpenReplicationStreamResponse, PersistFailure, PersistFailureReason, PersistRequest, + PersistResponse, PersistSuccess, ReplicateFailureReason, ReplicateSubrequest, + RetainShardsForSource, RetainShardsRequest, RetainShardsResponse, SynReplicationMessage, + TruncateShardsRequest, TruncateShardsResponse, }; use quickwit_proto::ingest::{ CommitTypeV2, DocBatchV2, IngestV2Error, IngestV2Result, ParseFailure, Shard, ShardIds, @@ -132,7 +130,7 @@ impl Ingester { pub async fn try_new( cluster: Cluster, control_plane: ControlPlaneServiceClient, - ingester_pool: Pool, + ingester_pool: IngesterPool, wal_dir_path: &Path, disk_capacity: ByteSize, memory_capacity: ByteSize, @@ -141,7 +139,7 @@ impl Ingester { idle_shard_timeout: Duration, ) -> IngestV2Result { let self_node_id: NodeId = cluster.self_node_id().into(); - let state = IngesterState::load(wal_dir_path, rate_limiter_settings); + let state = IngesterState::load(cluster.clone(), wal_dir_path, rate_limiter_settings).await; let weak_state = state.weak(); BroadcastLocalShardsTask::spawn(cluster, weak_state.clone()); @@ -164,12 +162,12 @@ impl Ingester { } /// Checks whether the ingester is fully decommissioned and updates its status accordingly. - fn check_decommissioning_status(&self, state: &mut InnerIngesterState) { + async fn check_decommissioning_status(&self, state: &mut InnerIngesterState) { if state.status() != IngesterStatus::Decommissioning { return; } if state.shards.values().all(|shard| shard.is_indexed()) { - state.set_status(IngesterStatus::Decommissioned); + state.set_status(IngesterStatus::Decommissioned).await; } } @@ -398,6 +396,7 @@ impl Ingester { IngestV2Error::Unavailable(message) })?; let mut ack_replication_stream = ingester + .client .open_replication_stream(syn_replication_stream) .await?; ack_replication_stream @@ -454,30 +453,17 @@ impl Ingester { let commit_type = persist_request.commit_type(); let force_commit = commit_type == CommitTypeV2::Force; - let leader_id: NodeId = persist_request.leader_id.into(); let mut state_guard = with_lock_metrics!(self.state.lock_fully().await, "persist", "write")?; - - if state_guard.status() != IngesterStatus::Ready { - persist_failures.reserve_exact(persist_request.subrequests.len()); - - for subrequest in persist_request.subrequests { - let persist_failure = PersistFailure { - subrequest_id: subrequest.subrequest_id, - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - reason: PersistFailureReason::ShardClosed as i32, - }; - persist_failures.push(persist_failure); - } - let persist_response = PersistResponse { - leader_id: leader_id.into(), - successes: Vec::new(), - failures: persist_failures, - }; - return Ok(persist_response); + let status = state_guard.status(); + + if !status.accepts_write_requests() { + let error = IngestV2Error::Unavailable(format!( + "ingester {} is not ready: {status}", + self.self_node_id + )); + return Err(error); } // first verify if we would locally accept each subrequest { @@ -844,9 +830,13 @@ impl Ingester { let follower_id: NodeId = open_replication_stream_request.follower_id.into(); let mut state_guard = self.state.lock_partially().await?; + let status = state_guard.status(); - if state_guard.status() != IngesterStatus::Ready { - return Err(IngestV2Error::Internal("node decommissioned".to_string())); + if !status.accepts_write_requests() { + let error = IngestV2Error::Unavailable(format!( + "ingester {follower_id} is not ready: {status}", + )); + return Err(error); } let Entry::Vacant(entry) = state_guard.replication_tasks.entry(leader_id.clone()) else { return Err(IngestV2Error::Internal(format!( @@ -929,9 +919,14 @@ impl Ingester { ) -> IngestV2Result { let mut state_guard = with_lock_metrics!(self.state.lock_fully().await, "init_shards", "write")?; - - if state_guard.status() != IngesterStatus::Ready { - return Err(IngestV2Error::Internal("node decommissioned".to_string())); + let status = state_guard.status(); + + if !status.accepts_write_requests() { + let error = IngestV2Error::Unavailable(format!( + "ingester {} is not ready: {status}", + self.self_node_id + )); + return Err(error); } let mut successes = Vec::with_capacity(init_shards_request.subrequests.len()); let mut failures = Vec::new(); @@ -1000,7 +995,7 @@ impl Ingester { let wal_usage = state_guard.mrecordlog.resource_usage(); report_wal_usage(wal_usage); - self.check_decommissioning_status(&mut state_guard); + self.check_decommissioning_status(&mut state_guard).await; let truncate_response = TruncateShardsResponse {}; Ok(truncate_response) } @@ -1027,22 +1022,6 @@ impl Ingester { Ok(response) } - async fn decommission_inner( - &self, - _decommission_request: DecommissionRequest, - ) -> IngestV2Result { - info!("decommissioning ingester"); - let mut state_guard = self.state.lock_partially().await?; - - for shard in state_guard.shards.values_mut() { - shard.close(); - } - state_guard.set_status(IngesterStatus::Decommissioning); - self.check_decommissioning_status(&mut state_guard); - - Ok(DecommissionResponse {}) - } - pub async fn debug_info(&self) -> JsonValue { let state_guard = match self.state.lock_fully().await { Ok(state_guard) => state_guard, @@ -1179,7 +1158,7 @@ impl IngesterService for Ingester { .delete_shard(&queue_id, "control-plane-retain-shards-rpc") .await; } - self.check_decommissioning_status(&mut state_guard); + self.check_decommissioning_status(&mut state_guard).await; Ok(RetainShardsResponse {}) } @@ -1199,9 +1178,50 @@ impl IngesterService for Ingester { async fn decommission( &self, - decommission_request: DecommissionRequest, + _decommission_request: DecommissionRequest, ) -> IngestV2Result { - self.decommission_inner(decommission_request).await + // Retire the ingester immediately by setting its status to `Retiring`. + info!("retiring ingester"); + let mut state_guard = self.state.lock_partially().await?; + state_guard.set_status(IngesterStatus::Retiring).await; + drop(state_guard); // Dropping explicitly for readability. + + // Drain write requests by scheduling the decommissioning of the ingester after a delay + // allowing the propagation of the `Retiring` status to other nodes. + let self_clone = self.clone(); + tokio::spawn(async move { + const DECOMMISSION_DELAY: Duration = if cfg!(any(test, feature = "testsuite")) { + Duration::from_millis(100) + } else { + // Having to wait for 10s is not great but we can live with it. During this time, we + // still make progress towards decommissioning because we gradually receive less + // write requests and indexing is still ongoing. However, it sets a floor on the + // amount of time with which we can fully decommission an ingester. This will be + // most noticeable when using Quickwit locally. + Duration::from_secs(10) + }; + tokio::time::sleep(DECOMMISSION_DELAY).await; + + info!("decommissioning ingester"); + let mut state_guard = match self_clone.state.lock_partially().await { + Ok(state_guard) => state_guard, + Err(error) => { + error!(%error, "failed to decommission ingester"); + return; + } + }; + state_guard + .set_status(IngesterStatus::Decommissioning) + .await; + + for shard in state_guard.shards.values_mut() { + shard.close(); + } + self_clone + .check_decommissioning_status(&mut state_guard) + .await; + }); + Ok(DecommissionResponse {}) } } @@ -1234,43 +1254,6 @@ impl EventSubscriber for WeakIngesterState { } } -pub async fn wait_for_ingester_status( - ingester: impl IngesterService, - status: IngesterStatus, -) -> anyhow::Result<()> { - let mut observation_stream = ingester - .open_observation_stream(OpenObservationStreamRequest {}) - .await - .context("failed to open observation stream")?; - - while let Some(observation_message_result) = observation_stream.next().await { - let observation_message = - observation_message_result.context("observation stream ended unexpectedly")?; - - if observation_message.status() == status { - break; - } - } - Ok(()) -} - -pub async fn wait_for_ingester_decommission(ingester: Ingester) -> anyhow::Result<()> { - let now = Instant::now(); - - ingester - .decommission(DecommissionRequest {}) - .await - .context("failed to initiate ingester decommission")?; - - wait_for_ingester_status(ingester, IngesterStatus::Decommissioned).await?; - - info!( - "successfully decommissioned ingester in {}", - now.elapsed().pretty_display() - ); - Ok(()) -} - struct PendingPersistSubrequest { queue_id: QueueId, subrequest_id: u32, @@ -1298,8 +1281,8 @@ mod tests { use quickwit_config::service::QuickwitService; use quickwit_proto::control_plane::{AdviseResetShardsResponse, MockControlPlaneService}; use quickwit_proto::ingest::ingester::{ - IngesterServiceGrpcServer, IngesterServiceGrpcServerAdapter, InitShardSubrequest, - PersistSubrequest, TruncateShardsSubrequest, + IngesterServiceClient, IngesterServiceGrpcServer, IngesterServiceGrpcServerAdapter, + IngesterStatus, InitShardSubrequest, PersistSubrequest, TruncateShardsSubrequest, }; use quickwit_proto::ingest::{ DocBatchV2, ParseFailureReason, ShardIdPosition, ShardIdPositions, ShardIds, ShardPKey, @@ -1312,10 +1295,11 @@ mod tests { use super::*; use crate::MRecord; - use crate::ingest_v2::DEFAULT_IDLE_SHARD_TIMEOUT; use crate::ingest_v2::broadcast::ShardInfos; use crate::ingest_v2::doc_mapper::try_build_doc_mapper; use crate::ingest_v2::fetch::tests::{into_fetch_eof, into_fetch_payload}; + use crate::ingest_v2::helpers::wait_for_ingester_status; + use crate::ingest_v2::{DEFAULT_IDLE_SHARD_TIMEOUT, IngesterPoolEntry}; const MAX_GRPC_MESSAGE_SIZE: ByteSize = ByteSize::mib(1); @@ -1431,7 +1415,7 @@ mod tests { .await .unwrap(); - wait_for_ingester_status(ingester.clone(), IngesterStatus::Ready) + wait_for_ingester_status(&ingester, IngesterStatus::Ready, Duration::from_secs(1)) .await .unwrap(); @@ -1460,9 +1444,10 @@ mod tests { let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); - let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); - let queue_id_03 = queue_id(&index_uid, "test-source", &ShardId::from(3)); + let source_id = SourceId::from("test-source"); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); + let queue_id_02 = queue_id(&index_uid, &source_id, &ShardId::from(2)); + let queue_id_03 = queue_id(&index_uid, &source_id, &ShardId::from(3)); state_guard .mrecordlog @@ -1514,7 +1499,7 @@ mod tests { .await .unwrap(); - state_guard.set_status(IngesterStatus::Initializing); + state_guard.set_status(IngesterStatus::Initializing).await; drop(state_guard); @@ -1546,22 +1531,15 @@ mod tests { let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); - let shard_00 = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(0), - ) - .build(); + let shard_00 = + IngesterShard::new_solo(index_uid.clone(), source_id.clone(), ShardId::from(0)).build(); state_guard.shards.insert(shard_00.queue_id(), shard_00); - let shard_01 = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .advertisable() - .build(); + let shard_01 = IngesterShard::new_solo(index_uid.clone(), source_id, ShardId::from(1)) + .advertisable() + .build(); let queue_id_01 = shard_01.queue_id(); state_guard.shards.insert(queue_id_01.clone(), shard_01); drop(state_guard); @@ -1615,6 +1593,7 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -1628,7 +1607,7 @@ mod tests { ); let primary_shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1649,7 +1628,7 @@ mod tests { .await .unwrap(); - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let shard = state_guard.shards.get(&queue_id).unwrap(); shard.assert_is_solo(); shard.assert_is_open(); @@ -1663,6 +1642,7 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -1672,7 +1652,7 @@ mod tests { ); let shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1700,7 +1680,7 @@ mod tests { let state_guard = ingester.state.lock_fully().await.unwrap(); - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let shard = state_guard.shards.get(&queue_id).unwrap(); shard.assert_is_solo(); shard.assert_is_open(); @@ -1714,8 +1694,9 @@ mod tests { async fn test_ingester_persist() { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; - let index_uid = IndexUid::for_test("test-index", 0); - let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + let index_uid_0 = IndexUid::for_test("test-index", 0); + let index_uid_1 = IndexUid::for_test("test-index", 1); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -1728,8 +1709,8 @@ mod tests { InitShardSubrequest { subrequest_id: 0, shard: Some(Shard { - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + index_uid: Some(index_uid_0.clone()), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1742,8 +1723,8 @@ mod tests { InitShardSubrequest { subrequest_id: 1, shard: Some(Shard { - index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + index_uid: Some(index_uid_1.clone()), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1763,15 +1744,15 @@ mod tests { subrequests: vec![ PersistSubrequest { subrequest_id: 0, - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + index_uid: Some(index_uid_0.clone()), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, - index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + index_uid: Some(index_uid_1.clone()), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([ r#"{"doc": "test-doc-110"}"#, @@ -1787,7 +1768,7 @@ mod tests { let persist_success_0 = &persist_response.successes[0]; assert_eq!(persist_success_0.subrequest_id, 0); - assert_eq!(persist_success_0.index_uid(), &index_uid); + assert_eq!(persist_success_0.index_uid(), &index_uid_0); assert_eq!(persist_success_0.source_id, "test-source"); assert_eq!(persist_success_0.shard_id(), ShardId::from(1)); assert_eq!( @@ -1797,7 +1778,7 @@ mod tests { let persist_success_1 = &persist_response.successes[1]; assert_eq!(persist_success_1.subrequest_id, 1); - assert_eq!(persist_success_1.index_uid(), &index_uid2); + assert_eq!(persist_success_1.index_uid(), &index_uid_1); assert_eq!(persist_success_1.source_id, "test-source"); assert_eq!(persist_success_1.shard_id(), ShardId::from(1)); assert_eq!( @@ -1808,7 +1789,7 @@ mod tests { let state_guard = ingester.state.lock_fully().await.unwrap(); assert_eq!(state_guard.shards.len(), 2); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid_0, &source_id, &ShardId::from(1)); let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); solo_shard_01.assert_is_solo(); solo_shard_01.assert_is_open(); @@ -1820,7 +1801,7 @@ mod tests { &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#), (1, [0, 1], "")], ); - let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); + let queue_id_11 = queue_id(&index_uid_1, &source_id, &ShardId::from(1)); let solo_shard_11 = state_guard.shards.get(&queue_id_11).unwrap(); solo_shard_11.assert_is_solo(); solo_shard_11.assert_is_open(); @@ -1842,6 +1823,7 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -1854,7 +1836,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1885,7 +1867,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(0)), doc_batch: None, }], @@ -1911,6 +1893,7 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -1925,7 +1908,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -1946,7 +1929,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(0)), doc_batch: Some(DocBatchV2::for_test([ "", // invalid @@ -1986,6 +1969,7 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2000,7 +1984,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -2021,7 +2005,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(0)), doc_batch: Some(DocBatchV2::for_test([ "", // invalid @@ -2049,6 +2033,7 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2063,7 +2048,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -2084,7 +2069,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(0)), doc_batch: Some(DocBatchV2::for_test(["", "[]", r#"{"foo": "bar"}"#])), }], @@ -2110,6 +2095,7 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2124,7 +2110,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -2145,7 +2131,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(0)), doc_batch: Some(DocBatchV2::for_test(["", "[]", r#"{"foo": "bar"}"#])), }], @@ -2177,13 +2163,10 @@ mod tests { let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); - let solo_shard = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .build(); + let source_id = SourceId::from("test-source"); + let solo_shard = + IngesterShard::new_solo(index_uid.clone(), source_id, ShardId::from(1)).build(); + let queue_id = solo_shard.queue_id(); state_guard.shards.insert(queue_id.clone(), solo_shard); state_guard @@ -2206,7 +2189,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id, shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"test-doc-foo"#])), }], @@ -2236,17 +2219,15 @@ mod tests { let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapper = try_build_doc_mapper("{}").unwrap(); // Insert a dangling shard, i.e. a shard without a corresponding queue. - let solo_shard = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .with_doc_mapper(doc_mapper) - .build(); + let solo_shard = + IngesterShard::new_solo(index_uid.clone(), source_id.clone(), ShardId::from(1)) + .with_doc_mapper(doc_mapper) + .build(); state_guard.shards.insert(solo_shard.queue_id(), solo_shard); drop(state_guard); @@ -2256,7 +2237,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-foo"}"#])), }], @@ -2297,11 +2278,15 @@ mod tests { leader_ctx.ingester_pool.insert( follower_ctx.node_id.clone(), - IngesterServiceClient::new(follower.clone()), + IngesterPoolEntry { + client: IngesterServiceClient::new(follower.clone()), + status: IngesterStatus::Ready, + }, ); let index_uid = IndexUid::for_test("test-index", 0); - let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + let index_uid2 = IndexUid::for_test("test-index", 1); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2315,7 +2300,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), @@ -2330,7 +2315,7 @@ mod tests { subrequest_id: 1, shard: Some(Shard { index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), @@ -2352,14 +2337,14 @@ mod tests { PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([ r#"{"doc": "test-doc-110"}"#, @@ -2396,7 +2381,7 @@ mod tests { let leader_state_guard = leader.state.lock_fully().await.unwrap(); assert_eq!(leader_state_guard.shards.len(), 2); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let primary_shard_01 = leader_state_guard.shards.get(&queue_id_01).unwrap(); primary_shard_01.assert_is_primary(); primary_shard_01.assert_is_open(); @@ -2408,7 +2393,7 @@ mod tests { &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#), (1, [0, 1], "")], ); - let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); + let queue_id_11 = queue_id(&index_uid2, &source_id, &ShardId::from(1)); let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); primary_shard_11.assert_is_primary(); primary_shard_11.assert_is_open(); @@ -2497,19 +2482,23 @@ mod tests { } }); let follower_channel = Endpoint::from_static("http://127.0.0.1:7777").connect_lazy(); - let follower_grpc_client = IngesterServiceClient::from_channel( + let follower_client = IngesterServiceClient::from_channel( "127.0.0.1:7777".parse().unwrap(), follower_channel, MAX_GRPC_MESSAGE_SIZE, None, ); - + let follower_ingester = IngesterPoolEntry { + client: follower_client, + status: IngesterStatus::Ready, + }; leader_ctx .ingester_pool - .insert(follower_ctx.node_id.clone(), follower_grpc_client); + .insert(follower_ctx.node_id.clone(), follower_ingester); let index_uid = IndexUid::for_test("test-index", 0); - let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); + let index_uid2 = IndexUid::for_test("test-index", 1); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2523,7 +2512,7 @@ mod tests { subrequest_id: 0, shard: Some(Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), @@ -2538,7 +2527,7 @@ mod tests { subrequest_id: 1, shard: Some(Shard { index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: leader_ctx.node_id.to_string(), @@ -2560,14 +2549,14 @@ mod tests { PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([ r#"{"doc": "test-doc-110"}"#, @@ -2604,7 +2593,7 @@ mod tests { let leader_state_guard = leader.state.lock_fully().await.unwrap(); assert_eq!(leader_state_guard.shards.len(), 2); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let primary_shard_01 = leader_state_guard.shards.get(&queue_id_01).unwrap(); primary_shard_01.assert_is_primary(); primary_shard_01.assert_is_open(); @@ -2616,7 +2605,7 @@ mod tests { &[(0, [0, 0], r#"{"doc": "test-doc-010"}"#)], ); - let queue_id_11 = queue_id(&index_uid2, "test-source", &ShardId::from(1)); + let queue_id_11 = queue_id(&index_uid2, &source_id, &ShardId::from(1)); let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); primary_shard_11.assert_is_primary(); primary_shard_11.assert_is_open(); @@ -2664,13 +2653,11 @@ mod tests { async fn test_ingester_persist_no_available_shards() { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); - let solo_shard = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .with_state(ShardState::Closed) - .build(); + let source_id = SourceId::from("test-source"); + let solo_shard = + IngesterShard::new_solo(index_uid.clone(), source_id.clone(), ShardId::from(1)) + .with_state(ShardState::Closed) + .build(); let queue_id = solo_shard.queue_id(); ingester .state @@ -2686,7 +2673,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], @@ -2727,6 +2714,7 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2736,7 +2724,7 @@ mod tests { ); let primary_shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -2765,7 +2753,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], @@ -2788,7 +2776,7 @@ mod tests { let state_guard = ingester.state.lock_fully().await.unwrap(); assert_eq!(state_guard.shards.len(), 1); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); solo_shard_01.assert_is_solo(); @@ -2808,6 +2796,7 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -2817,7 +2806,7 @@ mod tests { ); let primary_shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: ingester_ctx.node_id.to_string(), @@ -2846,7 +2835,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], @@ -2866,7 +2855,7 @@ mod tests { let state_guard = ingester.state.lock_fully().await.unwrap(); assert_eq!(state_guard.shards.len(), 1); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); solo_shard_01.assert_is_solo(); solo_shard_01.assert_is_open(); @@ -2916,10 +2905,11 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let open_fetch_stream_request = OpenFetchStreamRequest { client_id: "test-client".to_string(), index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1337)), from_position_exclusive: Some(Position::Beginning), }; @@ -2939,13 +2929,13 @@ mod tests { ); let shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let mut state_guard = ingester.state.lock_fully().await.unwrap(); @@ -2974,7 +2964,7 @@ mod tests { let open_fetch_stream_request = OpenFetchStreamRequest { client_id: "test-client".to_string(), index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1)), from_position_exclusive: Some(Position::Beginning), }; @@ -3039,8 +3029,9 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); - let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); + let source_id = SourceId::from("test-source"); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); + let queue_id_02 = queue_id(&index_uid, &source_id, &ShardId::from(2)); let doc_mapping_uid_01 = DocMappingUid::random(); let doc_mapping_json_01 = format!( @@ -3050,7 +3041,7 @@ mod tests { ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid_01), @@ -3065,7 +3056,7 @@ mod tests { ); let shard_02 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, doc_mapping_uid: Some(doc_mapping_uid_02), @@ -3127,19 +3118,19 @@ mod tests { subrequests: vec![ TruncateShardsSubrequest { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), truncate_up_to_position_inclusive: Some(Position::offset(0u64)), }, TruncateShardsSubrequest { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), truncate_up_to_position_inclusive: Some(Position::eof(0u64)), }, TruncateShardsSubrequest { index_uid: Some(IndexUid::for_test("test-index", 1337)), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1337)), truncate_up_to_position_inclusive: Some(Position::offset(1337u64)), }, @@ -3174,14 +3165,11 @@ mod tests { let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let mut state_guard = ingester.state.lock_fully().await.unwrap(); - let solo_shard = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .build(); + let solo_shard = + IngesterShard::new_solo(index_uid.clone(), source_id.clone(), ShardId::from(1)).build(); state_guard.shards.insert(solo_shard.queue_id(), solo_shard); drop(state_guard); @@ -3189,7 +3177,7 @@ mod tests { ingester_id: ingester_ctx.node_id.to_string(), subrequests: vec![TruncateShardsSubrequest { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1)), truncate_up_to_position_inclusive: Some(Position::offset(0u64)), }], @@ -3249,6 +3237,7 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3258,7 +3247,7 @@ mod tests { ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3266,13 +3255,13 @@ mod tests { }; let shard_02 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); + let queue_id_02 = queue_id(&index_uid, &source_id, &ShardId::from(2)); let mut state_guard = ingester.state.lock_fully().await.unwrap(); let now = Instant::now(); @@ -3328,6 +3317,7 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3337,7 +3327,7 @@ mod tests { ); let shard_17 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(17)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3346,7 +3336,7 @@ mod tests { let shard_18 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(18)), shard_state: ShardState::Closed as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3395,7 +3385,7 @@ mod tests { let retain_shards_request = RetainShardsRequest { retain_shards_for_sources: vec![RetainShardsForSource { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_ids: vec![ShardId::from(17u64)], }], }; @@ -3413,7 +3403,8 @@ mod tests { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid = IndexUid::for_test("test-index", 0); - let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let source_id = SourceId::from("test-source"); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3423,7 +3414,7 @@ mod tests { ); let shard = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3447,7 +3438,7 @@ mod tests { let open_fetch_stream_request = OpenFetchStreamRequest { client_id: "test-client".to_string(), index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), from_position_exclusive: Some(Position::Beginning), }; @@ -3460,12 +3451,12 @@ mod tests { shard_pkeys: vec![ ShardPKey { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), }, ShardPKey { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1337)), }, ], @@ -3514,7 +3505,9 @@ mod tests { assert_eq!(observation.status(), IngesterStatus::Ready); let mut state_guard = ingester.state.lock_fully().await.unwrap(); - state_guard.set_status(IngesterStatus::Decommissioning); + state_guard + .set_status(IngesterStatus::Decommissioning) + .await; drop(state_guard); let observation = observation_stream.next().await.unwrap().unwrap(); @@ -3527,40 +3520,87 @@ mod tests { assert!(observation_opt.is_none()); } + #[tokio::test] + async fn test_ingester_decommission() { + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; + + let mut state_guard = ingester.state.lock_fully().await.unwrap(); + let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); + + let shard = IngesterShard::new_solo(index_uid, source_id, ShardId::from(1)).build(); + let queue_id = shard.queue_id(); + + state_guard.shards.insert(queue_id.clone(), shard); + drop(state_guard); + + let mut observation_stream = ingester + .open_observation_stream(OpenObservationStreamRequest {}) + .await + .unwrap(); + + ingester.decommission(DecommissionRequest {}).await.unwrap(); + + let next_observation = observation_stream.next().await.unwrap().unwrap(); + let next_status = next_observation.status(); + assert_eq!(next_status, IngesterStatus::Retiring); + + wait_for_ingester_status( + &ingester, + IngesterStatus::Decommissioning, + Duration::from_secs(1), + ) + .await + .unwrap(); + + let state_guard = ingester.state.lock_fully().await.unwrap(); + let shard = state_guard.shards.get(&queue_id).unwrap(); + shard.assert_is_closed(); + } + #[tokio::test] async fn test_check_decommissioning_status() { let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let mut state_guard = ingester.state.lock_fully().await.unwrap(); - ingester.check_decommissioning_status(&mut state_guard); + ingester + .check_decommissioning_status(&mut state_guard) + .await; assert_eq!(state_guard.status(), IngesterStatus::Ready); - state_guard.set_status(IngesterStatus::Decommissioning); - ingester.check_decommissioning_status(&mut state_guard); + state_guard + .set_status(IngesterStatus::Decommissioning) + .await; + ingester + .check_decommissioning_status(&mut state_guard) + .await; assert_eq!(state_guard.status(), IngesterStatus::Decommissioned); - state_guard.set_status(IngesterStatus::Decommissioning); + state_guard + .set_status(IngesterStatus::Decommissioning) + .await; let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); - let solo_shard = IngesterShard::new_solo( - index_uid.clone(), - "test-source".to_string(), - ShardId::from(1), - ) - .with_state(ShardState::Closed) - .with_replication_position_inclusive(Position::offset(12u64)) - .build(); + let solo_shard = IngesterShard::new_solo(index_uid.clone(), source_id, ShardId::from(1)) + .with_state(ShardState::Closed) + .with_replication_position_inclusive(Position::offset(12u64)) + .build(); let queue_id = solo_shard.queue_id(); state_guard.shards.insert(queue_id.clone(), solo_shard); - ingester.check_decommissioning_status(&mut state_guard); + ingester + .check_decommissioning_status(&mut state_guard) + .await; assert_eq!(state_guard.status(), IngesterStatus::Decommissioning); let shard = state_guard.shards.get_mut(&queue_id).unwrap(); shard.truncation_position_inclusive = Position::Beginning.as_eof(); - ingester.check_decommissioning_status(&mut state_guard); + ingester + .check_decommissioning_status(&mut state_guard) + .await; assert_eq!(state_guard.status(), IngesterStatus::Decommissioned); } @@ -3571,6 +3611,7 @@ mod tests { ingester.subscribe(&event_broker); let index_uid = IndexUid::for_test("test-index", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3580,23 +3621,23 @@ mod tests { ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let shard_02 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, doc_mapping_uid: Some(doc_mapping_uid), ..Default::default() }; - let queue_id_02 = queue_id(&index_uid, "test-source", &ShardId::from(2)); + let queue_id_02 = queue_id(&index_uid, &source_id, &ShardId::from(2)); let mut state_guard = ingester.state.lock_fully().await.unwrap(); let now = Instant::now(); @@ -3649,7 +3690,7 @@ mod tests { let shard_position_update = ShardPositionsUpdate { source_uid: SourceUid { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id, }, updated_shard_positions: vec![ (ShardId::from(1), Position::offset(0u64)), @@ -3689,7 +3730,8 @@ mod tests { .await; let index_uid = IndexUid::for_test("test-index", 0); - let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); + let source_id = SourceId::from("test-source"); + let queue_id_01 = queue_id(&index_uid, &source_id, &ShardId::from(1)); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3699,7 +3741,7 @@ mod tests { ); let shard_01 = Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3742,6 +3784,7 @@ mod tests { let index_uid_0: IndexUid = IndexUid::for_test("test-index-0", 0); let index_uid_1: IndexUid = IndexUid::for_test("test-index-1", 0); + let source_id = SourceId::from("test-source"); let doc_mapping_uid = DocMappingUid::random(); let doc_mapping_json = format!( @@ -3751,7 +3794,7 @@ mod tests { ); let shard_01 = Shard { index_uid: Some(index_uid_0.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3759,7 +3802,7 @@ mod tests { }; let shard_02 = Shard { index_uid: Some(index_uid_0.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, doc_mapping_uid: Some(doc_mapping_uid), @@ -3767,7 +3810,7 @@ mod tests { }; let shard_03 = Shard { index_uid: Some(index_uid_1.clone()), - source_id: "test-source".to_string(), + source_id, shard_id: Some(ShardId::from(3)), shard_state: ShardState::Closed as i32, doc_mapping_uid: Some(doc_mapping_uid), diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index c8543faf793..977b50581b7 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -16,6 +16,7 @@ mod broadcast; mod debouncing; mod doc_mapper; mod fetch; +mod helpers; mod idle; mod ingester; mod metrics; @@ -41,7 +42,7 @@ use bytes::buf::Writer; use bytes::{BufMut, BytesMut}; use bytesize::ByteSize; use quickwit_common::tower::Pool; -use quickwit_proto::ingest::ingester::IngesterServiceClient; +use quickwit_proto::ingest::ingester::{IngesterServiceClient, IngesterStatus}; use quickwit_proto::ingest::router::{IngestRequestV2, IngestSubrequest}; use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2}; use quickwit_proto::types::{DocUid, DocUidGenerator, IndexId, NodeId, SubrequestId}; @@ -50,12 +51,21 @@ use tracing::{error, info}; use workbench::pending_subrequests; pub use self::fetch::{FetchStreamError, MultiFetchStream}; -pub use self::ingester::{Ingester, wait_for_ingester_decommission, wait_for_ingester_status}; +pub use self::helpers::{ + try_get_ingester_status, wait_for_ingester_decommission, wait_for_ingester_status, +}; +pub use self::ingester::Ingester; use self::mrecord::MRECORD_HEADER_LEN; pub use self::mrecord::{MRecord, decoded_mrecords}; pub use self::router::IngestRouter; -pub type IngesterPool = Pool; +#[derive(Clone)] +pub struct IngesterPoolEntry { + pub client: IngesterServiceClient, + pub status: IngesterStatus, +} + +pub type IngesterPool = Pool; /// Identifies an ingester client, typically a source, for logging and debugging purposes. pub type ClientId = String; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 0071f685044..bbf0cd037c5 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -761,6 +761,8 @@ impl Drop for ReplicationTaskHandle { #[cfg(test)] mod tests { + use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; + use quickwit_config::service::QuickwitService; use quickwit_proto::ingest::ingester::{ReplicateSubrequest, ReplicateSuccess}; use quickwit_proto::ingest::{DocBatchV2, Shard}; use quickwit_proto::types::{IndexUid, Position, ShardId, queue_id}; @@ -1036,7 +1038,15 @@ mod tests { async fn test_replication_task_happy_path() { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(SYN_REPLICATION_STREAM_CAPACITY); let (ack_replication_stream_tx, mut ack_replication_stream) = @@ -1299,7 +1309,15 @@ mod tests { async fn test_replication_task_shard_closed() { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(SYN_REPLICATION_STREAM_CAPACITY); let (ack_replication_stream_tx, mut ack_replication_stream) = @@ -1376,7 +1394,15 @@ mod tests { async fn test_replication_task_deletes_dangling_shard() { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(SYN_REPLICATION_STREAM_CAPACITY); let (ack_replication_stream_tx, mut ack_replication_stream) = @@ -1464,7 +1490,15 @@ mod tests { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(SYN_REPLICATION_STREAM_CAPACITY); let (ack_replication_stream_tx, mut ack_replication_stream) = @@ -1553,7 +1587,15 @@ mod tests { async fn test_replication_task_resource_exhausted() { let leader_id: NodeId = "test-leader".into(); let follower_id: NodeId = "test-follower".into(); - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(SYN_REPLICATION_STREAM_CAPACITY); let (ack_replication_stream_tx, mut ack_replication_stream) = diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 67ad31a2722..12a031960ab 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -434,7 +434,7 @@ impl IngestRouter { let persist_future = async move { let persist_result = tokio::time::timeout( PERSIST_REQUEST_TIMEOUT, - ingester.persist(persist_request), + ingester.client.persist(persist_request), ) .await .unwrap_or_else(|_| { @@ -709,7 +709,8 @@ mod tests { GetOrCreateOpenShardsResponse, GetOrCreateOpenShardsSuccess, MockControlPlaneService, }; use quickwit_proto::ingest::ingester::{ - IngesterServiceClient, MockIngesterService, PersistFailure, PersistResponse, PersistSuccess, + IngesterServiceClient, IngesterStatus, MockIngesterService, PersistFailure, + PersistResponse, PersistSuccess, }; use quickwit_proto::ingest::router::IngestSubrequest; use quickwit_proto::ingest::{ @@ -720,6 +721,7 @@ mod tests { use super::*; use crate::RateMibPerSec; + use crate::ingest_v2::IngesterPoolEntry; use crate::ingest_v2::broadcast::ShardInfo; use crate::ingest_v2::routing_table::{RoutingEntry, RoutingTableEntry}; use crate::ingest_v2::workbench::SubworkbenchFailure; @@ -841,7 +843,12 @@ mod tests { drop(rendezvous_1); drop(rendezvous_2); - ingester_pool.insert("test-ingester-0".into(), IngesterServiceClient::mocked()); + let client_0 = IngesterServiceClient::mocked(); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); { // Ingester-0 has been marked as unavailable due to the previous requests. let (get_or_create_open_shard_request_opt, _rendezvous) = router @@ -1359,8 +1366,19 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); - ingester_pool.insert("test-ingester-0".into(), IngesterServiceClient::mocked()); - ingester_pool.insert("test-ingester-1".into(), IngesterServiceClient::mocked()); + let client_0 = IngesterServiceClient::mocked(); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-0".into(), ingester_0); + + let client_1 = IngesterServiceClient::mocked(); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; + ingester_pool.insert("test-ingester-1".into(), ingester_1); let replication_factor = 1; let router = IngestRouter::new( @@ -1589,7 +1607,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let mut mock_ingester_1 = MockIngesterService::new(); @@ -1626,7 +1648,11 @@ mod tests { }; Ok(response) }); - let ingester_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); + let ingester_1 = IngesterPoolEntry { + client: client_1, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-1".into(), ingester_1); let ingest_request = IngestRequestV2 { @@ -1774,7 +1800,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let ingest_request = IngestRequestV2 { @@ -2077,7 +2107,11 @@ mod tests { }) .in_sequence(&mut seq); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let ingest_request = IngestRequestV2 { @@ -2153,7 +2187,11 @@ mod tests { }; Ok(response) }); - let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); + let ingester_0 = IngesterPoolEntry { + client: client_0, + status: IngesterStatus::Ready, + }; ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let ingest_request = IngestRequestV2 { diff --git a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs index 987d754ed69..3eef8788f64 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs @@ -126,12 +126,17 @@ impl RoutingTableEntry { if unavailable_leaders.contains(&shard.leader_id) { continue; } - if ingester_pool.contains_key(&shard.leader_id) { - return true; - } else { + let Some(ingester) = ingester_pool.get(&shard.leader_id) else { let leader_id: NodeId = shard.leader_id.clone(); unavailable_leaders.insert(leader_id); + continue; + }; + if !ingester.status.is_ready() { + let leader_id: NodeId = shard.leader_id.clone(); + unavailable_leaders.insert(leader_id); + continue; } + return true; } } } @@ -164,7 +169,9 @@ impl RoutingTableEntry { error = NextOpenShardError::RateLimited; continue; } - if ingester_pool.contains_key(&shard_routing_entry.leader_id) { + if let Some(ingester) = ingester_pool.get(&shard_routing_entry.leader_id) + && ingester.status.is_ready() + { return Ok(shard_routing_entry); } } @@ -497,15 +504,16 @@ impl RoutingTable { #[cfg(test)] mod tests { use quickwit_proto::ingest::ShardState; - use quickwit_proto::ingest::ingester::IngesterServiceClient; + use quickwit_proto::ingest::ingester::{IngesterServiceClient, IngesterStatus}; use super::*; + use crate::IngesterPoolEntry; #[test] fn test_routing_table_entry_new() { let self_node_id: NodeId = "test-node-0".into(); let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let table_entry = RoutingTableEntry::new( &self_node_id, index_uid.clone(), @@ -518,7 +526,7 @@ mod tests { let shards = vec![ Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(3)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), @@ -526,7 +534,7 @@ mod tests { }, Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), @@ -534,7 +542,7 @@ mod tests { }, Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, leader_id: "test-node-1".to_string(), @@ -542,7 +550,7 @@ mod tests { }, Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), @@ -550,7 +558,7 @@ mod tests { }, Shard { index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: Some(ShardId::from(4)), shard_state: ShardState::Closed as i32, leader_id: "test-node-0".to_string(), @@ -569,7 +577,7 @@ mod tests { #[test] fn test_routing_table_entry_has_open_shards() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); let mut closed_shard_ids = Vec::new(); @@ -584,88 +592,114 @@ mod tests { assert!(closed_shard_ids.is_empty()); assert!(unavailable_leaders.is_empty()); - ingester_pool.insert("test-ingester-0".into(), IngesterServiceClient::mocked()); - ingester_pool.insert("test-ingester-1".into(), IngesterServiceClient::mocked()); - + // Ingester 0 is ready, but shard 0 is closed. + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); + // Ingester 1 is ready, and shard 1 is open, but it is declared as unavailable by the caller + // via `unavailable_leaders`. + ingester_pool.insert( + "test-ingester-1".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); + // Ingester 2 is unavailable, i.e. not in the ingester pool. + // Ingester 3 is retiring. + ingester_pool.insert( + "test-ingester-3".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Retiring, + }, + ); + // Ingester 4 is ready, and shard 4 is open. + ingester_pool.insert( + "test-ingester-4".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); let table_entry = RoutingTableEntry { index_uid: index_uid.clone(), source_id: source_id.clone(), local_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - shard_id: ShardId::from(1), + source_id: source_id.clone(), + shard_id: ShardId::from(0), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - shard_id: ShardId::from(2), - shard_state: ShardState::Open, - leader_id: "test-ingester-0".into(), - }, - ], - local_round_robin_idx: AtomicUsize::default(), - remote_shards: Vec::new(), - remote_round_robin_idx: AtomicUsize::default(), - }; - assert!(table_entry.has_open_shards( - &ingester_pool, - &mut closed_shard_ids, - &mut unavailable_leaders - )); - assert_eq!(closed_shard_ids.len(), 1); - assert_eq!(closed_shard_ids[0], ShardId::from(1)); - assert!(unavailable_leaders.is_empty()); - - closed_shard_ids.clear(); - - let table_entry = RoutingTableEntry { - index_uid: index_uid.clone(), - source_id, - local_shards: Vec::new(), - local_round_robin_idx: AtomicUsize::default(), - remote_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), - shard_state: ShardState::Closed, + shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-2".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(3), shard_state: ShardState::Open, - leader_id: "test-ingester-1".into(), + leader_id: "test-ingester-3".into(), + }, + RoutingEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: ShardId::from(4), + shard_state: ShardState::Open, + leader_id: "test-ingester-4".into(), + }, + RoutingEntry { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: ShardId::from(4), + shard_state: ShardState::Open, + leader_id: "test-ingester-4".into(), }, ], + local_round_robin_idx: AtomicUsize::default(), + remote_shards: Vec::new(), remote_round_robin_idx: AtomicUsize::default(), }; + unavailable_leaders.insert("test-ingester-1".into()); + assert!(table_entry.has_open_shards( &ingester_pool, &mut closed_shard_ids, &mut unavailable_leaders )); - assert_eq!(closed_shard_ids.len(), 1); - assert_eq!(closed_shard_ids[0], ShardId::from(1)); - assert_eq!(unavailable_leaders.len(), 1); - assert!(unavailable_leaders.contains("test-ingester-2")); + assert_eq!(closed_shard_ids, vec![ShardId::from(0)]); + + assert_eq!( + unavailable_leaders, + HashSet::from_iter([ + "test-ingester-1".into(), + "test-ingester-2".into(), + "test-ingester-3".into(), + ]) + ); } #[test] fn test_routing_table_entry_next_open_shard_round_robin() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); let ingester_pool = IngesterPool::default(); let mut rate_limited_shards = HashSet::new(); @@ -675,8 +709,20 @@ mod tests { .unwrap_err(); assert_eq!(error, NextOpenShardError::NoShardsAvailable); - ingester_pool.insert("test-ingester-0".into(), IngesterServiceClient::mocked()); - ingester_pool.insert("test-ingester-1".into(), IngesterServiceClient::mocked()); + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); + ingester_pool.insert( + "test-ingester-1".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); let table_entry = RoutingTableEntry { index_uid: index_uid.clone(), @@ -684,21 +730,21 @@ mod tests { local_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), @@ -728,7 +774,7 @@ mod tests { source_id: source_id.clone(), local_shards: vec![RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), @@ -737,28 +783,28 @@ mod tests { remote_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(3), shard_state: ShardState::Closed, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(4), shard_state: ShardState::Open, leader_id: "test-ingester-2".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), @@ -792,10 +838,16 @@ mod tests { #[test] fn test_routing_table_entry_next_open_shard_round_robin_rate_limited_error() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let ingester_pool = IngesterPool::default(); - ingester_pool.insert("test-ingester-0".into(), IngesterServiceClient::mocked()); + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + }, + ); let rate_limited_shards = HashSet::from_iter([ShardId::from(1)]); @@ -804,7 +856,7 @@ mod tests { source_id: source_id.clone(), local_shards: vec![RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), @@ -822,7 +874,7 @@ mod tests { #[test] fn test_routing_table_entry_insert_open_shards() { let index_uid_0 = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let mut table_entry = RoutingTableEntry::empty(index_uid_0.clone(), source_id.clone()); let local_node_id: NodeId = "test-ingester-0".into(); @@ -931,7 +983,7 @@ mod tests { #[test] fn test_routing_table_entry_close_shards() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let mut table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); table_entry.close_shards(&index_uid, &[]); @@ -945,21 +997,21 @@ mod tests { local_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), @@ -969,21 +1021,21 @@ mod tests { remote_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(6), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(7), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), @@ -1012,7 +1064,7 @@ mod tests { #[test] fn test_routing_table_entry_delete_shards() { let index_uid = IndexUid::for_test("test-index", 0); - let source_id: SourceId = "test-source".into(); + let source_id = SourceId::from("test-source"); let mut table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); table_entry.delete_shards(&index_uid, &[]); @@ -1026,21 +1078,21 @@ mod tests { local_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(1), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), @@ -1050,21 +1102,21 @@ mod tests { remote_shards: vec![ RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(6), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + source_id: source_id.clone(), shard_id: ShardId::from(7), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), diff --git a/quickwit/quickwit-ingest/src/ingest_v2/state.rs b/quickwit/quickwit-ingest/src/ingest_v2/state.rs index 591ef4f704f..a19e4eb509d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/state.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/state.rs @@ -20,8 +20,10 @@ use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; use mrecordlog::error::{DeleteQueueError, TruncateError}; +use quickwit_cluster::Cluster; use quickwit_common::pretty::PrettyDisplay; use quickwit_common::rate_limiter::{RateLimiter, RateLimiterSettings}; +use quickwit_common::shared_consts::INGESTER_STATUS_KEY; use quickwit_doc_mapper::DocMapper; use quickwit_proto::control_plane::AdviseResetShardsResponse; use quickwit_proto::ingest::ingester::IngesterStatus; @@ -58,6 +60,7 @@ pub(super) struct InnerIngesterState { pub replication_streams: HashMap, // Replication tasks running for each replication stream opened with leaders. pub replication_tasks: HashMap, + cluster: Cluster, status: IngesterStatus, status_tx: watch::Sender, } @@ -67,9 +70,12 @@ impl InnerIngesterState { self.status } - pub fn set_status(&mut self, status: IngesterStatus) { + pub async fn set_status(&mut self, status: IngesterStatus) { self.status = status; self.status_tx.send(status).expect("channel should be open"); + self.cluster + .set_self_key_value(INGESTER_STATUS_KEY, status.as_json_str_name()) + .await; } /// Returns the shard with the most available permits for this index and source. @@ -90,17 +96,22 @@ impl InnerIngesterState { } impl IngesterState { - fn new() -> Self { + async fn create(cluster: Cluster) -> Self { let status = IngesterStatus::Initializing; let (status_tx, status_rx) = watch::channel(status); - let inner = InnerIngesterState { + let mut inner = InnerIngesterState { shards: Default::default(), doc_mappers: Default::default(), replication_streams: Default::default(), replication_tasks: Default::default(), + cluster, status, status_tx, }; + // We call `set_status` here instead of setting it directly because it also updates the + // ingester status in chitchat. + inner.set_status(IngesterStatus::Initializing).await; + let inner = Arc::new(Mutex::new(inner)); let mrecordlog = Arc::new(RwLock::new(None)); @@ -111,8 +122,12 @@ impl IngesterState { } } - pub fn load(wal_dir_path: &Path, rate_limiter_settings: RateLimiterSettings) -> Self { - let state = Self::new(); + pub async fn load( + cluster: Cluster, + wal_dir_path: &Path, + rate_limiter_settings: RateLimiterSettings, + ) -> Self { + let state = Self::create(cluster).await; let state_clone = state.clone(); let wal_dir_path = wal_dir_path.to_path_buf(); @@ -125,15 +140,12 @@ impl IngesterState { } #[cfg(test)] - pub async fn for_test() -> (tempfile::TempDir, Self) { + pub async fn for_test(cluster: Cluster) -> (tempfile::TempDir, Self) { let temp_dir = tempfile::tempdir().unwrap(); - let mut state = IngesterState::load(temp_dir.path(), RateLimiterSettings::default()); + let mut state = + IngesterState::load(cluster, temp_dir.path(), RateLimiterSettings::default()).await; - state - .status_rx - .wait_for(|status| *status == IngesterStatus::Ready) - .await - .unwrap(); + state.wait_for_ready().await; (temp_dir, state) } @@ -168,7 +180,7 @@ impl IngesterState { } Err(error) => { error!("failed to open WAL: {error}"); - inner_guard.set_status(IngesterStatus::Failed); + inner_guard.set_status(IngesterStatus::Failed).await; return; } }; @@ -229,7 +241,7 @@ impl IngesterState { info!("deleted {num_deleted_shards} empty shard(s)"); } mrecordlog_guard.replace(mrecordlog); - inner_guard.set_status(IngesterStatus::Ready); + inner_guard.set_status(IngesterStatus::Ready).await; } pub async fn wait_for_ready(&mut self) { @@ -386,38 +398,32 @@ impl FullyLockedIngesterState<'_> { truncate_up_to_position_inclusive: Position, initiator: &'static str, ) { - // TODO: Replace with if-let-chains when stabilized. - let Some(truncate_up_to_offset_inclusive) = truncate_up_to_position_inclusive.as_u64() - else { - return; - }; - let Some(shard) = self.inner.shards.get_mut(queue_id) else { - return; - }; - if shard.truncation_position_inclusive >= truncate_up_to_position_inclusive { - return; - } - match self - .mrecordlog - .truncate(queue_id, truncate_up_to_offset_inclusive) - .await + if let Some(truncate_up_to_offset_inclusive) = truncate_up_to_position_inclusive.as_u64() + && let Some(shard) = self.inner.shards.get_mut(queue_id) + && shard.truncation_position_inclusive < truncate_up_to_position_inclusive { - Ok(_) => { - info!( - "truncated shard `{queue_id}` at {truncate_up_to_position_inclusive} \ - initiated via `{initiator}`" - ); - shard.truncation_position_inclusive = truncate_up_to_position_inclusive; - } - Err(TruncateError::MissingQueue(_)) => { - error!("failed to truncate shard `{queue_id}`: WAL queue not found"); - self.shards.remove(queue_id); - info!("deleted dangling shard `{queue_id}`"); - } - Err(TruncateError::IoError(io_error)) => { - error!("failed to truncate shard `{queue_id}`: {io_error}"); - } - }; + match self + .mrecordlog + .truncate(queue_id, truncate_up_to_offset_inclusive) + .await + { + Ok(_) => { + info!( + "truncated shard `{queue_id}` at {truncate_up_to_position_inclusive} \ + initiated via `{initiator}`" + ); + shard.truncation_position_inclusive = truncate_up_to_position_inclusive; + } + Err(TruncateError::MissingQueue(_)) => { + error!("failed to truncate shard `{queue_id}`: WAL queue not found"); + self.shards.remove(queue_id); + info!("deleted dangling shard `{queue_id}`"); + } + Err(TruncateError::IoError(io_error)) => { + error!("failed to truncate shard `{queue_id}`: {io_error}"); + } + }; + } } /// Deletes and truncates the shards as directed by the `advise_reset_shards_response` returned @@ -467,6 +473,8 @@ impl WeakIngesterState { #[cfg(test)] mod tests { use bytesize::ByteSize; + use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; + use quickwit_config::service::QuickwitService; use quickwit_proto::types::ShardId; use tokio::time::timeout; @@ -474,7 +482,15 @@ mod tests { #[tokio::test] async fn test_ingester_state_does_not_lock_while_initializing() { - let state = IngesterState::new(); + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let state = IngesterState::create(cluster.clone()).await; let inner_guard = state.inner.lock().await; assert_eq!(inner_guard.status(), IngesterStatus::Initializing); @@ -489,9 +505,22 @@ mod tests { #[tokio::test] async fn test_ingester_state_failed() { - let state = IngesterState::new(); + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let state = IngesterState::create(cluster.clone()).await; - state.inner.lock().await.set_status(IngesterStatus::Failed); + state + .inner + .lock() + .await + .set_status(IngesterStatus::Failed) + .await; let error = state.lock_partially().await.unwrap_err().to_string(); assert!(error.to_string().ends_with("failed to initialize ingester")); @@ -502,7 +531,15 @@ mod tests { #[tokio::test] async fn test_ingester_state_init() { - let mut state = IngesterState::new(); + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let mut state = IngesterState::create(cluster.clone()).await; let temp_dir = tempfile::tempdir().unwrap(); state @@ -539,7 +576,15 @@ mod tests { #[tokio::test] async fn test_find_most_capacity_shard_returns_shard_with_least_used_capacity() { - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let mut state_guard = state.lock_partially().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); @@ -577,7 +622,15 @@ mod tests { #[tokio::test] async fn test_find_most_capacity_shard_skips_closed_shards() { - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let mut locked_state = state.lock_partially().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); @@ -619,7 +672,15 @@ mod tests { #[tokio::test] async fn test_find_most_capacity_shard_returns_none_for_unknown_index_or_source() { - let (_temp_dir, state) = IngesterState::for_test().await; + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster).await; let mut locked_state = state.lock_partially().await.unwrap(); let index_uid = IndexUid::for_test("test-index", 0); @@ -642,4 +703,34 @@ mod tests { locked_state.find_most_capacity_shard_mut(&index_uid, &SourceId::from("other-source")); assert!(shard_opt.is_none()); } + + #[tokio::test] + async fn test_ingester_state_set_status() { + let cluster = create_cluster_for_test( + Vec::new(), + &[QuickwitService::Indexer.as_str()], + &ChannelTransport::default(), + true, + ) + .await + .unwrap(); + let state = IngesterState::create(cluster.clone()).await; + let temp_dir = tempfile::tempdir().unwrap(); + + state + .init(temp_dir.path(), RateLimiterSettings::default()) + .await; + + let mut state_guard = state.lock_fully().await.unwrap(); + state_guard.set_status(IngesterStatus::Failed).await; + assert_eq!(state_guard.status(), IngesterStatus::Failed); + assert_eq!(*state.status_rx.borrow(), IngesterStatus::Failed); + + let status_json_str = cluster + .get_self_key_value(INGESTER_STATUS_KEY) + .await + .unwrap(); + let status = IngesterStatus::from_json_str_name(&status_json_str).unwrap(); + assert_eq!(status, IngesterStatus::Failed); + } } diff --git a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs index 5277e0ac740..c17d5ea5bfb 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs @@ -351,7 +351,7 @@ impl ClusterSandbox { TraceServiceClient::new(self.channel(QuickwitService::Indexer)) } - async fn wait_for_cluster_num_ready_nodes( + pub async fn wait_for_cluster_num_ready_nodes( &self, expected_num_ready_nodes: usize, ) -> anyhow::Result<()> { @@ -596,4 +596,18 @@ impl ClusterSandbox { self.shutdown_services(QuickwitService::supported_services()) .await } + + /// Remove a node from the sandbox and return its shutdown handle. + /// After this call, `rest_client` and other lookup methods skip the removed + /// node, so callers can trigger shutdown concurrently with other sandbox + /// operations. + pub fn remove_node_with_service(&mut self, service: QuickwitService) -> NodeShutdownHandle { + let idx = self + .node_shutdown_handles + .iter() + .position(|h| h.node_services.contains(&service)) + .unwrap_or_else(|| panic!("no node with service {service:?}")); + self.node_configs.remove(idx); + self.node_shutdown_handles.remove(idx) + } } diff --git a/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs b/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs index 4a5e29c9565..3169dfaf28a 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs @@ -856,3 +856,139 @@ async fn test_shutdown_indexer_first() { .unwrap() .unwrap(); } + +/// Tests that the graceful shutdown sequence works correctly in a multi-indexer +/// cluster: shutting down one indexer does NOT cause 500 errors or data loss, +/// and the cluster eventually rebalances. see #6158 +#[tokio::test] +async fn test_graceful_shutdown_no_data_loss() { + let mut sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::Indexer]) + .add_node([ + QuickwitService::ControlPlane, + QuickwitService::Searcher, + QuickwitService::Metastore, + QuickwitService::Janitor, + ]) + .build_and_start() + .await; + let index_id = "test_graceful_shutdown_no_data_loss"; + + // Create index with a long commit timeout so documents stay uncommitted + // in the ingesters' WAL. The decommission sequence should commit + // them before the indexer quits. + sandbox + .rest_client(QuickwitService::Indexer) + .indexes() + .create( + format!( + r#" + version: 0.8 + index_id: {index_id} + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 5 + "# + ), + ConfigFormat::Yaml, + false, + ) + .await + .unwrap(); + + // Ingest docs with auto-commit. With a 5s commit timeout, these documents + // sit uncommitted in the ingesters' WAL - exactly the in-flight state we + // want to exercise during draining. + ingest( + &sandbox.rest_client(QuickwitService::Indexer), + index_id, + ingest_json!({"body": "before-shutdown-1"}), + CommitType::Auto, + ) + .await + .unwrap(); + + ingest( + &sandbox.rest_client(QuickwitService::Indexer), + index_id, + ingest_json!({"body": "before-shutdown-2"}), + CommitType::Auto, + ) + .await + .unwrap(); + + // Remove the first indexer node from the sandbox and get its shutdown handle. + // After this call, rest_client(Indexer) returns the second (surviving) indexer. + let shutdown_handle = sandbox.remove_node_with_service(QuickwitService::Indexer); + + // Concurrently: shut down the removed indexer AND ingest more data via the + // surviving indexer. This verifies the cluster stays operational and the + // router on the surviving node does not return 500 errors while one indexer + // is decommissioning. The control plane excludes the decommissioning + // ingester from shard allocation, so new shards go to the surviving one. + let ingest_client = sandbox.rest_client(QuickwitService::Indexer); + let (shutdown_result, ingest_result) = tokio::join!( + async { + tokio::time::timeout(Duration::from_secs(30), shutdown_handle.shutdown()) + .await + .expect("indexer shutdown timed out — decommission may be stuck") + }, + async { + // Small delay so the decommission sequence has started before we ingest. + tokio::time::sleep(Duration::from_millis(200)).await; + ingest( + &ingest_client, + index_id, + ingest_json!({"body": "during-shutdown"}), + CommitType::Auto, + ) + .await + }, + ); + shutdown_result.expect("indexer shutdown failed"); + ingest_result.expect("ingest during shutdown should succeed (no 500 errors)"); + + // All 3 documents should eventually be searchable. Documents 1 & 2 were + // in-flight on the decommissioning indexer and should have been committed during + // the decommission step. Document 3 was ingested to the surviving indexer. + wait_until_predicate( + || async { + match sandbox + .rest_client(QuickwitService::Searcher) + .search( + index_id, + quickwit_serve::SearchRequestQueryString { + query: "*".to_string(), + max_hits: 10, + ..Default::default() + }, + ) + .await + { + Ok(resp) => resp.num_hits == 3, + Err(_) => false, + } + }, + Duration::from_secs(30), + Duration::from_millis(500), + ) + .await + .expect("expected 3 documents after decommission shutdown, some data may have been lost"); + + // Verify the cluster sees 2 ready nodes (the surviving indexer + the + // control-plane/searcher/metastore/janitor node). + sandbox + .wait_for_cluster_num_ready_nodes(2) + .await + .expect("cluster should see 2 ready nodes after indexer shutdown"); + + // Clean shutdown of the remaining nodes. + tokio::time::timeout(Duration::from_secs(15), sandbox.shutdown()) + .await + .unwrap() + .unwrap(); +} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index a5b651d94d8..f7341b12862 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -300,6 +300,8 @@ enum IngesterStatus { INGESTER_STATUS_INITIALIZING = 1; // The ingester is ready and accepts read and write requests. INGESTER_STATUS_READY = 2; + // The ingester is about to be decommissioned. It still accepts read and write requests, but will not accept write requests in a few seconds and should be avoided by future write requests. + INGESTER_STATUS_RETIRING = 6; // The ingester is being decommissioned. It accepts read requests but rejects write requests // (open shards, persist, and replicate requests). It will transition to `Decommissioned` once // all shards are fully indexed. 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 018e19a39a9..6ed35eacdb0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -475,6 +475,8 @@ pub enum IngesterStatus { Initializing = 1, /// The ingester is ready and accepts read and write requests. Ready = 2, + /// The ingester is about to be decommissioned. It still accepts read and write requests, but will not accept write requests in a few seconds and should be avoided by future write requests. + Retiring = 6, /// The ingester is being decommissioned. It accepts read requests but rejects write requests /// (open shards, persist, and replicate requests). It will transition to `Decommissioned` once /// all shards are fully indexed. @@ -495,6 +497,7 @@ impl IngesterStatus { Self::Unspecified => "INGESTER_STATUS_UNSPECIFIED", Self::Initializing => "INGESTER_STATUS_INITIALIZING", Self::Ready => "INGESTER_STATUS_READY", + Self::Retiring => "INGESTER_STATUS_RETIRING", Self::Decommissioning => "INGESTER_STATUS_DECOMMISSIONING", Self::Decommissioned => "INGESTER_STATUS_DECOMMISSIONED", Self::Failed => "INGESTER_STATUS_FAILED", @@ -506,6 +509,7 @@ impl IngesterStatus { "INGESTER_STATUS_UNSPECIFIED" => Some(Self::Unspecified), "INGESTER_STATUS_INITIALIZING" => Some(Self::Initializing), "INGESTER_STATUS_READY" => Some(Self::Ready), + "INGESTER_STATUS_RETIRING" => Some(Self::Retiring), "INGESTER_STATUS_DECOMMISSIONING" => Some(Self::Decommissioning), "INGESTER_STATUS_DECOMMISSIONED" => Some(Self::Decommissioned), "INGESTER_STATUS_FAILED" => Some(Self::Failed), diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs index d2da3f8d9bd..0236380f8f5 100644 --- a/quickwit/quickwit-proto/src/ingest/ingester.rs +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -72,11 +72,39 @@ impl IngesterStatus { Self::Unspecified => "unspecified", Self::Initializing => "initializing", Self::Ready => "ready", + Self::Retiring => "retiring", Self::Decommissioning => "decommissioning", Self::Decommissioned => "decommissioned", Self::Failed => "failed", } } + + pub fn from_json_str_name(value: &str) -> Option { + match value { + "unspecified" => Some(Self::Unspecified), + "initializing" => Some(Self::Initializing), + "ready" => Some(Self::Ready), + "retiring" => Some(Self::Retiring), + "decommissioning" => Some(Self::Decommissioning), + "decommissioned" => Some(Self::Decommissioned), + "failed" => Some(Self::Failed), + _ => None, + } + } + + pub fn is_ready(&self) -> bool { + matches!(self, Self::Ready) + } + + pub fn accepts_write_requests(&self) -> bool { + matches!(self, Self::Ready | Self::Retiring) + } +} + +impl std::fmt::Display for IngesterStatus { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.as_json_str_name()) + } } impl OpenFetchStreamRequest { diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 16f8a36002d..8671b52b737 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -47,6 +47,7 @@ tokio = { workspace = true } tokio-rustls = { workspace = true } tokio-stream = { workspace = true } tokio-util = { workspace = true } +tonic = { workspace = true } tonic-health = { workspace = true } tonic-reflection = { workspace = true } tower = { workspace = true, features = ["limit"] } diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index dccaecd3912..3dc5f80d604 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -59,7 +59,7 @@ use itertools::Itertools; use once_cell::sync::Lazy; use quickwit_actors::{ActorExitStatus, Mailbox, SpawnContext, Universe}; use quickwit_cluster::{ - Cluster, ClusterChange, ClusterChangeStream, ListenerHandle, start_cluster_service, + Cluster, ClusterChange, ClusterChangeStream, ClusterNode, ListenerHandle, start_cluster_service, }; use quickwit_common::pubsub::{EventBroker, EventSubscriptionHandle}; use quickwit_common::rate_limiter::RateLimiterSettings; @@ -82,8 +82,9 @@ use quickwit_indexing::models::ShardPositionsService; use quickwit_indexing::start_indexing_service; use quickwit_ingest::{ GetMemoryCapacity, IngestRequest, IngestRouter, IngestServiceClient, Ingester, IngesterPool, - LocalShardsUpdate, get_idle_shard_timeout, setup_local_shards_update_listener, - start_ingest_api_service, wait_for_ingester_decommission, wait_for_ingester_status, + IngesterPoolEntry, LocalShardsUpdate, get_idle_shard_timeout, + setup_local_shards_update_listener, start_ingest_api_service, try_get_ingester_status, + wait_for_ingester_decommission, wait_for_ingester_status, }; use quickwit_jaeger::JaegerService; use quickwit_janitor::{JanitorService, start_janitor_service}; @@ -112,6 +113,7 @@ use quickwit_search::{ use quickwit_storage::{SplitCache, StorageResolver}; use tcp_listener::TcpListenerResolver; use tokio::sync::oneshot; +use tonic::codec::CompressionEncoding; use tonic_health::ServingStatus; use tonic_health::server::HealthReporter; use tower::ServiceBuilder; @@ -379,8 +381,8 @@ fn start_shard_positions_service( // We spawn a task here, because we need the ingester to be ready before spawning the // the `ShardPositionsService`. If we don't, all the events we emit too early will be dismissed. tokio::spawn(async move { - if let Some(ingester) = ingester_opt - && wait_for_ingester_status(ingester, IngesterStatus::Ready) + if let Some(ingester) = &ingester_opt + && wait_for_ingester_status(ingester, IngesterStatus::Ready, Duration::from_secs(300)) .await .is_err() { @@ -406,8 +408,8 @@ async fn shutdown_signal_handler( shutdown_signal.await; // We must decommission the ingester first before terminating the indexing pipelines that // may consume from it. We also need to keep the gRPC server running while doing so. - if let Some(ingester) = ingester_opt - && let Err(error) = wait_for_ingester_decommission(ingester).await + if let Some(ingester) = &ingester_opt + && let Err(error) = wait_for_ingester_decommission(ingester, Duration::from_secs(300)).await { error!("failed to decommission ingester gracefully: {:?}", error); } @@ -556,10 +558,10 @@ pub async fn serve_quickwit( // Setup the indexer pool to track cluster changes. setup_indexer_pool( - &node_config, cluster.change_stream(), - indexer_pool, indexing_service_opt.clone(), + indexer_pool, + node_config.grpc_config.max_message_size, ); // Setup ingest service v2. @@ -963,64 +965,141 @@ async fn setup_ingest_v2( } else { None }; - // Setup ingester pool change stream. - let ingester_opt_clone = ingester_opt.clone(); - let max_message_size = node_config.grpc_config.max_message_size; - let ingester_change_stream = cluster.change_stream().filter_map(move |cluster_change| { - let ingester_opt_clone_clone = ingester_opt_clone.clone(); + setup_ingester_pool( + cluster.change_stream(), + ingester_opt.clone(), + ingester_pool, + grpc_compression_encoding_opt, + node_config.grpc_config.max_message_size, + ); + Ok((ingest_router, ingest_router_service, ingester_opt)) +} + +fn setup_ingester_pool( + cluster_change_stream: ClusterChangeStream, + ingester_opt: Option, + ingester_pool: IngesterPool, + grpc_compression_encoding_opt: Option, + grpc_max_message_size: ByteSize, +) { + let ingester_change_stream = cluster_change_stream.filter_map(move |cluster_change| { + let ingester_opt_clone = ingester_opt.clone(); Box::pin(async move { match cluster_change { - ClusterChange::Add(node) if node.is_indexer() => { - let chitchat_id = node.chitchat_id(); - info!( - node_id = chitchat_id.node_id, - generation_id = chitchat_id.generation_id, - "adding node `{}` to ingester pool", - chitchat_id.node_id, + ClusterChange::Add(node) + if node.is_indexer() && node.ingester_status().is_ready() => + { + let change = build_ingester_insert_change( + &node, + ingester_opt_clone, + grpc_max_message_size, + grpc_compression_encoding_opt, ); - let node_id: NodeId = node.node_id().into(); - - if node.is_self_node() { - // Here, since the service is available locally, we bypass the network stack - // and use the instance directly. However, we still want client-side - // metrics, so we use both metrics layers. - let ingester = ingester_opt_clone_clone - .expect("ingester service should be initialized"); - let ingester_service = ingester_service_layer_stack( - IngesterServiceClient::tower() - .stack_layer(INGEST_GRPC_CLIENT_METRICS_LAYER.clone()), - ) - .build(ingester); - Some(Change::Insert(node_id, ingester_service)) + Some(change) + } + ClusterChange::Update { previous, updated } if updated.is_indexer() => { + if !previous.is_ready() && updated.is_ready() { + let change = build_ingester_insert_change( + &updated, + ingester_opt_clone, + grpc_max_message_size, + grpc_compression_encoding_opt, + ); + Some(change) + } else if previous.is_ready() && !updated.is_ready() { + let change = build_ingester_remove_change(&previous); + Some(change) + } else if previous.ingester_status().is_ready() + && !updated.ingester_status().is_ready() + { + let change = build_ingester_insert_change( + &updated, + ingester_opt_clone, + grpc_max_message_size, + grpc_compression_encoding_opt, + ); + Some(change) } else { - let ingester_service = IngesterServiceClient::tower() - .stack_layer(INGEST_GRPC_CLIENT_METRICS_LAYER.clone()) - .stack_layer(TimeoutLayer::new(GRPC_INGESTER_SERVICE_TIMEOUT)) - .build_from_channel( - node.grpc_advertise_addr(), - node.channel(), - max_message_size, - grpc_compression_encoding_opt, - ); - Some(Change::Insert(node_id, ingester_service)) + None } } ClusterChange::Remove(node) if node.is_indexer() => { - let chitchat_id = node.chitchat_id(); - info!( - node_id = chitchat_id.node_id, - generation_id = chitchat_id.generation_id, - "removing node `{}` from ingester pool", - chitchat_id.node_id, - ); - Some(Change::Remove(node.node_id().into())) + let change = build_ingester_remove_change(&node); + Some(change) } _ => None, } }) }); ingester_pool.listen_for_changes(ingester_change_stream); - Ok((ingest_router, ingest_router_service, ingester_opt)) +} + +fn build_ingester_insert_change( + node: &ClusterNode, + ingester_opt: Option, + grpc_max_message_size: ByteSize, + grpc_compression_encoding_opt: Option, +) -> Change { + let chitchat_id = node.chitchat_id(); + info!( + node_id = chitchat_id.node_id, + generation_id = chitchat_id.generation_id, + "adding/updating node `{}` with ingester status `{}` to ingester pool", + chitchat_id.node_id, + node.ingester_status(), + ); + let node_id: NodeId = node.node_id().into(); + let ingester_service = build_ingester_service( + node, + ingester_opt, + grpc_max_message_size, + grpc_compression_encoding_opt, + ); + let pool_entry = IngesterPoolEntry { + client: ingester_service, + status: node.ingester_status(), + }; + Change::Insert(node_id, pool_entry) +} + +fn build_ingester_remove_change(node: &ClusterNode) -> Change { + let chitchat_id = node.chitchat_id(); + info!( + node_id = chitchat_id.node_id, + generation_id = chitchat_id.generation_id, + "removing node `{}` from ingester pool", + chitchat_id.node_id, + ); + let node_id: NodeId = node.node_id().into(); + Change::Remove(node_id) +} + +fn build_ingester_service( + node: &ClusterNode, + ingester_opt: Option, + max_message_size: ByteSize, + grpc_compression_encoding_opt: Option, +) -> IngesterServiceClient { + if node.is_self_node() { + // Here, since the service is available locally, we bypass the network stack + // and use the instance directly. However, we still want client-side + // metrics, so we use both metrics layers. + let ingester = ingester_opt.expect("ingester service should be initialized"); + let service = ingester_service_layer_stack( + IngesterServiceClient::tower().stack_layer(INGEST_GRPC_CLIENT_METRICS_LAYER.clone()), + ) + .build(ingester); + return service; + } + IngesterServiceClient::tower() + .stack_layer(INGEST_GRPC_CLIENT_METRICS_LAYER.clone()) + .stack_layer(TimeoutLayer::new(GRPC_INGESTER_SERVICE_TIMEOUT)) + .build_from_channel( + node.grpc_advertise_addr(), + node.channel(), + max_message_size, + grpc_compression_encoding_opt, + ) } async fn setup_searcher( @@ -1144,90 +1223,28 @@ async fn setup_control_plane( } fn setup_indexer_pool( - node_config: &NodeConfig, cluster_change_stream: ClusterChangeStream, - indexer_pool: IndexerPool, indexing_service_opt: Option>, + indexer_pool: IndexerPool, + grpc_max_message_size: ByteSize, ) { - let max_message_size = node_config.grpc_config.max_message_size; let indexer_change_stream = cluster_change_stream.filter_map(move |cluster_change| { let indexing_service_clone_opt = indexing_service_opt.clone(); Box::pin(async move { - match &cluster_change { - ClusterChange::Add(node) if node.is_indexer() => { - let chitchat_id = node.chitchat_id(); - info!( - node_id = chitchat_id.node_id, - generation_id = chitchat_id.generation_id, - "adding node `{}` to indexer pool", - chitchat_id.node_id, - ); - } - _ => {} - }; match cluster_change { - ClusterChange::Add(node) | ClusterChange::Update(node) if node.is_indexer() => { - let node_id = node.node_id().to_owned(); - let indexing_tasks = node.indexing_tasks().to_vec(); - let indexing_capacity = node.indexing_capacity(); - - if node.is_self_node() { - // Here, since the service is available locally, we bypass the network stack - // and use the mailbox directly. However, we still want client-side metrics, - // so we use both metrics layers. - let indexing_service_mailbox = indexing_service_clone_opt - .expect("indexing service should be initialized"); - // These layers apply to all the RPCs of the indexing service. - let shared_layers = ServiceBuilder::new() - .layer(INDEXING_GRPC_CLIENT_METRICS_LAYER.clone()) - .layer(INDEXING_GRPC_SERVER_METRICS_LAYER.clone()) - .into_inner(); - let client = IndexingServiceClient::tower() - .stack_layer(shared_layers) - .build_from_mailbox(indexing_service_mailbox); - let change = Change::Insert( - node_id.clone(), - IndexerNodeInfo { - node_id, - generation_id: node.chitchat_id().generation_id, - client, - indexing_tasks, - indexing_capacity, - }, - ); - Some(change) - } else { - let client = IndexingServiceClient::tower() - .stack_layer(INDEXING_GRPC_CLIENT_METRICS_LAYER.clone()) - .stack_layer(TimeoutLayer::new(GRPC_INDEXING_SERVICE_TIMEOUT)) - .build_from_channel( - node.grpc_advertise_addr(), - node.channel(), - max_message_size, - None, - ); - let change = Change::Insert( - node_id.clone(), - IndexerNodeInfo { - node_id, - generation_id: node.chitchat_id().generation_id, - client, - indexing_tasks, - indexing_capacity, - }, - ); - Some(change) - } + ClusterChange::Add(node) | ClusterChange::Update { updated: node, .. } + if node.is_indexer() => + { + let change = build_indexer_insert_change( + &node, + indexing_service_clone_opt, + grpc_max_message_size, + ); + Some(change) } ClusterChange::Remove(node) if node.is_indexer() => { - let chitchat_id = node.chitchat_id(); - info!( - node_id = chitchat_id.node_id, - generation_id = chitchat_id.generation_id, - "removing node `{}` from indexer pool", - chitchat_id.node_id, - ); - Some(Change::Remove(node.node_id().to_owned())) + let change = build_indexer_remove_change(&node); + Some(change) } _ => None, } @@ -1236,6 +1253,74 @@ fn setup_indexer_pool( indexer_pool.listen_for_changes(indexer_change_stream); } +fn build_indexer_insert_change( + node: &ClusterNode, + indexing_service_opt: Option>, + grpc_max_message_size: ByteSize, +) -> Change { + let chitchat_id = node.chitchat_id(); + info!( + node_id = chitchat_id.node_id, + generation_id = chitchat_id.generation_id, + "adding node `{}` to indexer pool", + chitchat_id.node_id, + ); + let node_id: NodeId = node.node_id().into(); + let client = build_indexing_service(node, indexing_service_opt, grpc_max_message_size); + Change::Insert( + node_id.clone(), + IndexerNodeInfo { + node_id, + generation_id: chitchat_id.generation_id, + client, + indexing_tasks: node.indexing_tasks().to_vec(), + indexing_capacity: node.indexing_capacity(), + }, + ) +} + +fn build_indexer_remove_change(node: &ClusterNode) -> Change { + let chitchat_id = node.chitchat_id(); + info!( + node_id = chitchat_id.node_id, + generation_id = chitchat_id.generation_id, + "removing node `{}` from indexer pool", + chitchat_id.node_id, + ); + let node_id: NodeId = node.node_id().into(); + Change::Remove(node_id) +} + +fn build_indexing_service( + node: &ClusterNode, + indexing_service_opt: Option>, + max_message_size: ByteSize, +) -> IndexingServiceClient { + if node.is_self_node() { + // Here, since the service is available locally, we bypass the network stack + // and use the mailbox directly. However, we still want client-side metrics, + // so we use both metrics layers. + let indexing_service_mailbox = + indexing_service_opt.expect("indexing service should be initialized"); + let shared_layers = ServiceBuilder::new() + .layer(INDEXING_GRPC_CLIENT_METRICS_LAYER.clone()) + .layer(INDEXING_GRPC_SERVER_METRICS_LAYER.clone()) + .into_inner(); + return IndexingServiceClient::tower() + .stack_layer(shared_layers) + .build_from_mailbox(indexing_service_mailbox); + } + IndexingServiceClient::tower() + .stack_layer(INDEXING_GRPC_CLIENT_METRICS_LAYER.clone()) + .stack_layer(TimeoutLayer::new(GRPC_INDEXING_SERVICE_TIMEOUT)) + .build_from_channel( + node.grpc_advertise_addr(), + node.channel(), + max_message_size, + None, + ) +} + fn require( val_opt: Option, ) -> impl Filter + Clone { @@ -1284,28 +1369,48 @@ async fn node_readiness_reporting_task( }; info!("REST server is ready"); - if let Some(ingester) = ingester_opt - && let Err(error) = wait_for_ingester_status(ingester, IngesterStatus::Ready).await - { - error!("failed to initialize ingester: {:?}", error); - info!("shutting down"); - return; - } let mut interval = tokio::time::interval(READINESS_REPORTING_INTERVAL); loop { interval.tick().await; - let new_node_ready = match metastore.check_connectivity().await { + let metastore_is_available = match metastore.check_connectivity().await { Ok(()) => { debug!(metastore_endpoints=?metastore.endpoints(), "metastore service is available"); - true + if let Some(ingester) = &ingester_opt { + if let Ok(status) = try_get_ingester_status(ingester).await { + status != IngesterStatus::Failed + } else { + // If we couldn't get the ingester status, it's not looking good, so we set + // the node to not ready. + false + } + } else { + true + } } Err(error) => { warn!(metastore_endpoints=?metastore.endpoints(), error=?error, "metastore service is unavailable"); false } }; + let ingester_is_available = if let Some(ingester) = &ingester_opt { + match try_get_ingester_status(ingester).await { + Ok(status) => { + status == IngesterStatus::Initializing || status != IngesterStatus::Failed + } + Err(error) => { + // If we couldn't get the ingester status, it's not looking good, so we set the + // node to not ready. + error!(%error, "failed to get ingester status"); + false + } + } + } else { + true + }; + let new_node_ready = metastore_is_available && ingester_is_available; + if new_node_ready != node_ready { node_ready = new_node_ready; cluster.set_self_node_readiness(node_ready).await; @@ -1513,16 +1618,16 @@ mod tests { ClusterChangeStream::new_unbounded(); let indexer_pool = IndexerPool::default(); setup_indexer_pool( - &node_config, cluster_change_stream, - indexer_pool.clone(), Some(indexing_service_mailbox), + indexer_pool.clone(), + node_config.grpc_config.max_message_size, ); let new_indexer_node = ClusterNode::for_test("test-indexer-node", 1, true, &["indexer"], &[]).await; cluster_change_stream_tx - .send(ClusterChange::Add(new_indexer_node)) + .send(ClusterChange::Add(new_indexer_node.clone())) .unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; @@ -1547,7 +1652,10 @@ mod tests { ) .await; cluster_change_stream_tx - .send(ClusterChange::Update(updated_indexer_node.clone())) + .send(ClusterChange::Update { + previous: new_indexer_node, + updated: updated_indexer_node.clone(), + }) .unwrap(); tokio::time::sleep(Duration::from_millis(1)).await; @@ -1626,4 +1734,72 @@ mod tests { .unwrap(); assert!(!searcher_client.is_local()); } + + #[tokio::test] + async fn test_setup_ingester_pool() { + let (cluster_change_stream, cluster_change_stream_tx) = + ClusterChangeStream::new_unbounded(); + let ingester_pool = IngesterPool::default(); + setup_ingester_pool( + cluster_change_stream, + None::, + ingester_pool.clone(), + None, + ByteSize::mib(20), + ); + + // Add an indexer node with IngesterStatus::Ready. + let new_node = ClusterNode::for_test_with_ingester_status( + "test-ingester-node", + 1, + false, + &["indexer"], + &[], + IngesterStatus::Ready, + ) + .await; + cluster_change_stream_tx + .send(ClusterChange::Add(new_node.clone())) + .unwrap(); + tokio::time::sleep(Duration::from_millis(1)).await; + + assert_eq!(ingester_pool.len(), 1); + let pool_entry = ingester_pool + .get(&NodeId::from("test-ingester-node")) + .unwrap(); + assert_eq!(pool_entry.status, IngesterStatus::Ready); + + // Update the node: ingester status transitions from Ready to Decommissioning. + let updated_node = ClusterNode::for_test_with_ingester_status( + "test-ingester-node", + 1, + false, + &["indexer"], + &[], + IngesterStatus::Decommissioning, + ) + .await; + cluster_change_stream_tx + .send(ClusterChange::Update { + previous: new_node, + updated: updated_node.clone(), + }) + .unwrap(); + tokio::time::sleep(Duration::from_millis(1)).await; + + // The node should still be in the pool with updated status. + assert_eq!(ingester_pool.len(), 1); + let pool_entry = ingester_pool + .get(&NodeId::from("test-ingester-node")) + .unwrap(); + assert_eq!(pool_entry.status, IngesterStatus::Decommissioning); + + // Remove the node. + cluster_change_stream_tx + .send(ClusterChange::Remove(updated_node)) + .unwrap(); + tokio::time::sleep(Duration::from_millis(1)).await; + + assert!(ingester_pool.is_empty()); + } }