diff --git a/quickwit/quickwit-cluster/src/node.rs b/quickwit/quickwit-cluster/src/node.rs index b83e354c93d..fbdedbcc6b4 100644 --- a/quickwit/quickwit-cluster/src/node.rs +++ b/quickwit/quickwit-cluster/src/node.rs @@ -50,6 +50,7 @@ impl ClusterNode { ingester_status: member.ingester_status, is_ready: member.is_ready, is_self_node, + availability_zone: member.availability_zone, }; let node = ClusterNode { inner: Arc::new(inner), @@ -141,6 +142,10 @@ impl ClusterNode { pub fn is_self_node(&self) -> bool { self.inner.is_self_node } + + pub fn availability_zone(&self) -> Option<&str> { + self.inner.availability_zone.as_deref() + } } impl Debug for ClusterNode { @@ -162,6 +167,7 @@ impl PartialEq for ClusterNode { && self.inner.indexing_tasks == other.inner.indexing_tasks && self.inner.is_ready == other.inner.is_ready && self.inner.is_self_node == other.inner.is_self_node + && self.inner.availability_zone == other.inner.availability_zone } } @@ -175,4 +181,5 @@ struct InnerNode { ingester_status: IngesterStatus, is_ready: bool, is_self_node: bool, + availability_zone: Option, } diff --git a/quickwit/quickwit-common/src/lib.rs b/quickwit/quickwit-common/src/lib.rs index 0f3af2bc5ba..11147f975f9 100644 --- a/quickwit/quickwit-common/src/lib.rs +++ b/quickwit/quickwit-common/src/lib.rs @@ -36,6 +36,7 @@ pub mod rate_limited_tracing; pub mod rate_limiter; pub mod rendezvous_hasher; pub mod retry; +pub mod ring_buffer; pub mod runtimes; pub mod shared_consts; pub mod sorted_iter; diff --git a/quickwit/quickwit-common/src/ring_buffer.rs b/quickwit/quickwit-common/src/ring_buffer.rs new file mode 100644 index 00000000000..5d884d8188f --- /dev/null +++ b/quickwit/quickwit-common/src/ring_buffer.rs @@ -0,0 +1,170 @@ +// 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::fmt::{Debug, Formatter}; + +/// Fixed-size buffer that keeps the last N elements pushed into it. +/// +/// `head` is the write cursor. It advances by one on each push and wraps +/// back to 0 when it reaches N, overwriting the oldest element. +/// +/// ```text +/// RingBuffer after pushing 1, 2, 3, 4, 5, 6: +/// +/// buffer = [5, 6, 3, 4] head = 2 len = 4 +/// ^ +/// next write goes here +/// +/// logical view (oldest → newest): [3, 4, 5, 6] +/// ``` +pub struct RingBuffer { + buffer: [T; N], + head: usize, + len: usize, +} + +impl Default for RingBuffer { + fn default() -> Self { + Self { + buffer: [T::default(); N], + head: 0, + len: 0, + } + } +} + +impl Debug for RingBuffer { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_list().entries(self.iter()).finish() + } +} + +impl RingBuffer { + pub fn push_back(&mut self, value: T) { + self.buffer[self.head] = value; + self.head = (self.head + 1) % N; + if self.len < N { + self.len += 1; + } + } + + pub fn last(&self) -> Option { + if self.len == 0 { + return None; + } + Some(self.buffer[(self.head + N - 1) % N]) + } + + pub fn front(&self) -> Option { + if self.len == 0 { + return None; + } + Some(self.buffer[(self.head + N - self.len) % N]) + } + + pub fn len(&self) -> usize { + self.len + } + + pub fn is_empty(&self) -> bool { + self.len == 0 + } + + pub fn iter(&self) -> impl Iterator + '_ { + let start = (self.head + N - self.len) % N; + (0..self.len).map(move |i| &self.buffer[(start + i) % N]) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_empty() { + let rb = RingBuffer::::default(); + assert!(rb.is_empty()); + assert_eq!(rb.len(), 0); + assert_eq!(rb.last(), None); + assert_eq!(rb.front(), None); + assert_eq!(rb.iter().count(), 0); + } + + #[test] + fn test_single_push() { + let mut rb = RingBuffer::::default(); + rb.push_back(10); + assert_eq!(rb.len(), 1); + assert!(!rb.is_empty()); + assert_eq!(rb.last(), Some(10)); + assert_eq!(rb.front(), Some(10)); + assert_eq!(rb.iter().copied().collect::>(), vec![10]); + } + + #[test] + fn test_partial_fill() { + let mut rb = RingBuffer::::default(); + rb.push_back(1); + rb.push_back(2); + rb.push_back(3); + assert_eq!(rb.len(), 3); + assert_eq!(rb.last(), Some(3)); + assert_eq!(rb.front(), Some(1)); + assert_eq!(rb.iter().copied().collect::>(), vec![1, 2, 3]); + } + + #[test] + fn test_exactly_full() { + let mut rb = RingBuffer::::default(); + for i in 1..=4 { + rb.push_back(i); + } + assert_eq!(rb.len(), 4); + assert_eq!(rb.last(), Some(4)); + assert_eq!(rb.front(), Some(1)); + assert_eq!(rb.iter().copied().collect::>(), vec![1, 2, 3, 4]); + } + + #[test] + fn test_wrap_around() { + let mut rb = RingBuffer::::default(); + for i in 1..=6 { + rb.push_back(i); + } + assert_eq!(rb.len(), 4); + assert_eq!(rb.last(), Some(6)); + assert_eq!(rb.front(), Some(3)); + assert_eq!(rb.iter().copied().collect::>(), vec![3, 4, 5, 6]); + } + + #[test] + fn test_many_wraps() { + let mut rb = RingBuffer::::default(); + for i in 1..=100 { + rb.push_back(i); + } + assert_eq!(rb.len(), 3); + assert_eq!(rb.last(), Some(100)); + assert_eq!(rb.front(), Some(98)); + assert_eq!(rb.iter().copied().collect::>(), vec![98, 99, 100]); + } + + #[test] + fn test_debug() { + let mut rb = RingBuffer::::default(); + rb.push_back(1); + rb.push_back(2); + assert_eq!(format!("{:?}", rb), "[1, 2]"); + } +} diff --git a/quickwit/quickwit-common/src/shared_consts.rs b/quickwit/quickwit-common/src/shared_consts.rs index d46cbc535ca..611636a5aa9 100644 --- a/quickwit/quickwit-common/src/shared_consts.rs +++ b/quickwit/quickwit-common/src/shared_consts.rs @@ -67,6 +67,9 @@ 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"; +/// Prefix used in chitchat to broadcast per-source ingester capacity scores and open shard counts. +pub const INGESTER_CAPACITY_SCORE_PREFIX: &str = "ingester.capacity_score:"; + /// 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-control-plane/Cargo.toml b/quickwit/quickwit-control-plane/Cargo.toml index 2957c9858c4..abaf672e908 100644 --- a/quickwit/quickwit-control-plane/Cargo.toml +++ b/quickwit/quickwit-control-plane/Cargo.toml @@ -47,6 +47,7 @@ quickwit-cluster = { workspace = true, features = ["testsuite"] } quickwit-common = { workspace = true, features = ["testsuite"] } quickwit-config = { workspace = true, features = ["testsuite"] } quickwit-indexing = { workspace = true } +quickwit-ingest = { workspace = true, features = ["testsuite"] } quickwit-metastore = { workspace = true, features = ["testsuite"] } quickwit-proto = { workspace = true, features = ["testsuite"] } diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 1185f8a7bbc..1056aba6eb8 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -2223,11 +2223,8 @@ mod tests { assert!(&retain_shards_for_source.shard_ids.is_empty()); Ok(RetainShardsResponse {}) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("node1".into(), ingester); let cluster_config = ClusterConfig::for_test(); @@ -2273,11 +2270,8 @@ mod tests { ); Ok(RetainShardsResponse {}) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("node1".into(), ingester); let mut index_0 = IndexMetadata::for_test("test-index-0", "ram:///test-index-0"); @@ -2652,11 +2646,8 @@ mod tests { }; Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert(ingester_id, ingester); let mut mock_metastore = MockMetastoreService::new(); @@ -2810,11 +2801,8 @@ mod tests { }; Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); 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 18c9c6486e1..53d3ee4034f 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -325,17 +325,17 @@ impl IngestController { /// Syncs the ingester in a fire and forget manner. /// /// The returned oneshot is just here for unit test to wait for the operation to terminate. - fn sync_with_ingester(&self, ingester: &NodeId, model: &ControlPlaneModel) -> WaitHandle { - info!(ingester = %ingester, "sync_with_ingester"); + fn sync_with_ingester(&self, ingester_id: &NodeId, model: &ControlPlaneModel) -> WaitHandle { + info!(ingester = %ingester_id, "sync_with_ingester"); let (wait_drop_guard, wait_handle) = WaitHandle::new(); - let Some(ingester_client) = self.ingester_pool.get(ingester) else { + let Some(ingester) = self.ingester_pool.get(ingester_id) else { // TODO: (Maybe) We should mark the ingester as unavailable, and stop advertise its // shard to routers. - warn!("failed to sync with ingester `{ingester}`: not available"); + warn!("failed to sync with ingester `{ingester_id}`: not available"); return wait_handle; }; let mut retain_shards_req = RetainShardsRequest::default(); - for (source_uid, shard_ids) in &*model.list_shards_for_node(ingester) { + for (source_uid, shard_ids) in &*model.list_shards_for_node(ingester_id) { let shards_for_source = RetainShardsForSource { index_uid: Some(source_uid.index_uid.clone()), source_id: source_uid.source_id.clone(), @@ -345,14 +345,12 @@ impl IngestController { .retain_shards_for_sources .push(shards_for_source); } - info!(ingester = %ingester, "retain shards ingester"); - let operation: String = format!("retain shards `{ingester}`"); + info!(%ingester_id, "retain shards ingester"); + let operation: String = format!("retain shards `{ingester_id}`"); fire_and_forget( async move { - if let Err(retain_shards_err) = ingester_client - .client - .retain_shards(retain_shards_req) - .await + if let Err(retain_shards_err) = + ingester.client.retain_shards(retain_shards_req).await { error!(%retain_shards_err, "retain shards error"); } @@ -1400,11 +1398,10 @@ mod tests { let client = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; - ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); + ingester_pool.insert( + NodeId::from("test-ingester-1"), + IngesterPoolEntry::ready_with_client(client.clone()), + ); let mut mock_ingester = MockIngesterService::new(); let index_uid_1_clone = index_uid_1.clone(); @@ -1431,12 +1428,11 @@ mod tests { }; Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; - ingester_pool.insert(NodeId::from("test-ingester-2"), ingester); + let ingester = IngesterServiceClient::from_mock(mock_ingester); + ingester_pool.insert( + NodeId::from("test-ingester-2"), + IngesterPoolEntry::ready_with_client(ingester.clone()), + ); let replication_factor = 2; let mut controller = IngestController::new( @@ -1622,11 +1618,10 @@ mod tests { let client = IngesterServiceClient::from_mock(mock_ingester); let ingester_pool = IngesterPool::default(); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; - ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); + ingester_pool.insert( + NodeId::from("test-ingester-1"), + IngesterPoolEntry::ready_with_client(client.clone()), + ); let replication_factor = 1; let mut controller = IngestController::new( @@ -1737,12 +1732,10 @@ mod tests { controller.allocate_shards(0, &FnvHashSet::default(), &model); assert!(leader_follower_pairs_opt.is_none()); - 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); + ingester_pool.insert( + NodeId::from("test-ingester-1"), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::mocked()), + ); let leader_follower_pairs_opt = controller.allocate_shards(0, &FnvHashSet::default(), &model); @@ -1751,12 +1744,10 @@ mod tests { // find any solution. assert!(leader_follower_pairs_opt.is_none()); - let client_2 = IngesterServiceClient::mocked(); - let ingester_2 = IngesterPoolEntry { - client: client_2, - status: IngesterStatus::Ready, - }; - ingester_pool.insert("test-ingester-2".into(), ingester_2); + ingester_pool.insert( + "test-ingester-2".into(), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::mocked()), + ); let leader_follower_pairs = controller .allocate_shards(0, &FnvHashSet::default(), &model) @@ -1875,12 +1866,10 @@ mod tests { Some(NodeId::from("test-ingester-1")) ); - let client = IngesterServiceClient::mocked(); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; - ingester_pool.insert("test-ingester-3".into(), ingester); + ingester_pool.insert( + "test-ingester-3".into(), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::mocked()), + ); let unavailable_leaders = FnvHashSet::from_iter([NodeId::from("test-ingester-2")]); let leader_follower_pairs = controller .allocate_shards(4, &unavailable_leaders, &model) @@ -1972,12 +1961,11 @@ mod tests { }; Ok(response) }); - 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_0 = IngesterServiceClient::from_mock(mock_ingester_0); + ingester_pool.insert( + ingester_id_0, + IngesterPoolEntry::ready_with_client(ingester_0), + ); let ingester_id_1 = NodeId::from("test-ingester-1"); let mut mock_ingester_1 = MockIngesterService::new(); @@ -1998,11 +1986,8 @@ mod tests { Err(IngestV2Error::Internal("internal error".to_string())) }); - let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); - let ingester_1 = IngesterPoolEntry { - client: client_1, - status: IngesterStatus::Ready, - }; + let ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); ingester_pool.insert(ingester_id_1, ingester_1); let ingester_id_2 = NodeId::from("test-ingester-2"); @@ -2012,11 +1997,10 @@ mod tests { 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); + ingester_pool.insert( + ingester_id_2, + IngesterPoolEntry::ready_with_client(client_2), + ); let init_shards_response = controller .init_shards(Vec::new(), &Progress::default()) @@ -2222,12 +2206,10 @@ mod tests { Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; - ingester_pool.insert(NodeId::from("test-ingester-1"), ingester); + ingester_pool.insert( + NodeId::from("test-ingester-1"), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)), + ); let source_uids: HashMap = HashMap::from_iter([(source_uid.clone(), 1)]); let unavailable_leaders = FnvHashSet::default(); let progress = Progress::default(); @@ -2396,11 +2378,8 @@ mod tests { "failed to close shards".to_string(), )) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("test-ingester".into(), ingester); let shard_infos = BTreeSet::from_iter([ @@ -2553,11 +2532,8 @@ mod tests { }, ); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("test-ingester".into(), ingester); let shard_infos = BTreeSet::from_iter([ShardInfo { @@ -2703,11 +2679,8 @@ mod tests { }; Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("test-ingester".into(), ingester); // Test failed to open shards. @@ -2829,11 +2802,8 @@ mod tests { }; Ok(response) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); ingester_pool.insert("test-ingester".into(), ingester); // Test failed to close shard. @@ -3063,24 +3033,18 @@ mod tests { count_calls_clone.fetch_add(1, Ordering::Release); Ok(RetainShardsResponse {}) }); - 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); + ingester_pool.insert( + "node-1".into(), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)), + ); + ingester_pool.insert( + "node-2".into(), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_2)), + ); + ingester_pool.insert( + "node-3".into(), + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_3)), + ); let node_id = "node-1".into(); let wait_handle = controller.sync_with_ingester(&node_id, &model); wait_handle.wait().await; @@ -3208,12 +3172,11 @@ mod tests { }; Ok(response) }); - 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_0 = IngesterServiceClient::from_mock(mock_ingester_0); + ingester_pool.insert( + ingester_id_0.clone(), + IngesterPoolEntry::ready_with_client(ingester_0), + ); let ingester_id_1 = NodeId::from("test-ingester-1"); let mut mock_ingester_1 = MockIngesterService::new(); @@ -3230,12 +3193,11 @@ mod tests { Err(IngestV2Error::Internal("internal error".to_string())) }); - 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_1 = IngesterServiceClient::from_mock(mock_ingester_1); + ingester_pool.insert( + ingester_id_1.clone(), + IngesterPoolEntry::ready_with_client(ingester_1), + ); let ingester_id_2 = NodeId::from("test-ingester-2"); let mut mock_ingester_2 = MockIngesterService::new(); @@ -3244,11 +3206,10 @@ mod tests { 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); + ingester_pool.insert( + ingester_id_2.clone(), + IngesterPoolEntry::ready_with_client(client_2), + ); // In this test: // - ingester 0 will close shard 0 successfully and fail to close shard 1; @@ -3427,12 +3388,11 @@ mod tests { }; Ok(response) }); - 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_0 = IngesterServiceClient::from_mock(mock_ingester_0); + ingester_pool.insert( + ingester_id_0.clone(), + IngesterPoolEntry::ready_with_client(ingester_0), + ); let ingester_id_1 = NodeId::from("test-ingester-1"); let mut mock_ingester_1 = MockIngesterService::new(); @@ -3473,11 +3433,8 @@ mod tests { }; Ok(response) }); - let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); - let ingester_1 = IngesterPoolEntry { - client: client_1, - status: IngesterStatus::Ready, - }; + let ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); ingester_pool.insert(ingester_id_1.clone(), ingester_1); let num_opened_shards = controller @@ -3665,6 +3622,7 @@ mod tests { let ingester = IngesterPoolEntry { client: ingester_client.clone(), status: IngesterStatus::Ready, + availability_zone: None, }; ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); } @@ -3681,6 +3639,7 @@ mod tests { let ingester = IngesterPoolEntry { client: ingester_client.clone(), status: IngesterStatus::Retiring, + availability_zone: None, }; ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); } @@ -3821,6 +3780,7 @@ mod tests { let ingester = IngesterPoolEntry { client: ingester_client.clone(), status: IngesterStatus::Decommissioned, + availability_zone: None, }; ingester_pool.insert(NodeId::from(ingester_id.clone()), ingester); } diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 273f2df5b37..46b7ddd8e0e 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -676,8 +676,7 @@ mod tests { use quickwit_ingest::IngesterPoolEntry; use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::ingest::ingester::{ - FetchMessage, IngesterServiceClient, IngesterStatus, MockIngesterService, - TruncateShardsResponse, + FetchMessage, IngesterServiceClient, MockIngesterService, TruncateShardsResponse, }; use quickwit_proto::ingest::{IngestV2Error, MRecordBatch, Shard, ShardState}; use quickwit_proto::metastore::{AcquireShardsResponse, MockMetastoreService}; @@ -932,12 +931,9 @@ mod tests { Ok(response) }); - 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 ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let event_broker = EventBroker::default(); @@ -1133,12 +1129,9 @@ mod tests { Ok(response) }); - 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 ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = @@ -1302,12 +1295,9 @@ mod tests { Ok(response) }); - 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 ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = @@ -1614,12 +1604,9 @@ mod tests { }) }); - 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 ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let event_broker = EventBroker::default(); let source_runtime = SourceRuntime { @@ -1718,12 +1705,9 @@ mod tests { Ok(TruncateShardsResponse {}) }); - 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 ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); + ingester_pool.insert("test-ingester-0".into(), ingester_0.clone()); let mut mock_ingester_1 = MockIngesterService::new(); mock_ingester_1 @@ -1749,12 +1733,9 @@ mod tests { Ok(TruncateShardsResponse {}) }); - 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 ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); + ingester_pool.insert("test-ingester-1".into(), ingester_1.clone()); let mut mock_ingester_3 = MockIngesterService::new(); mock_ingester_3 @@ -1773,12 +1754,9 @@ mod tests { Ok(TruncateShardsResponse {}) }); - 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 ingester_3 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_3)); + ingester_pool.insert("test-ingester-3".into(), ingester_3.clone()); let event_broker = EventBroker::default(); let (shard_positions_update_tx, mut shard_positions_update_rx) = diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index 3dfa0bf6c0c..3149f2aaaf3 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -18,6 +18,7 @@ bytesize = { workspace = true } fail = { workspace = true, optional = true } futures = { workspace = true } http = { workspace = true } +itertools = { workspace = true } mockall = { workspace = true, optional = true } mrecordlog = { workspace = true } once_cell = { workspace = true } @@ -43,7 +44,6 @@ quickwit-doc-mapper = { workspace = true, features = ["testsuite"] } quickwit-proto = { workspace = true } [dev-dependencies] -itertools = { workspace = true } mockall = { workspace = true } rand = { workspace = true } rand_distr = { workspace = true } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs new file mode 100644 index 00000000000..062539db797 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/capacity_score.rs @@ -0,0 +1,267 @@ +// 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::collections::BTreeSet; + +use anyhow::{Context, Result}; +use bytesize::ByteSize; +use quickwit_cluster::{Cluster, ListenerHandle}; +use quickwit_common::pubsub::{Event, EventBroker}; +use quickwit_common::shared_consts::INGESTER_CAPACITY_SCORE_PREFIX; +use quickwit_proto::ingest::ingester::IngesterStatus; +use quickwit_proto::types::{NodeId, SourceUid}; +use serde::{Deserialize, Serialize}; +use tokio::task::JoinHandle; +use tracing::{info, warn}; + +use super::{BROADCAST_INTERVAL_PERIOD, make_key, parse_key}; +use crate::OpenShardCounts; +use crate::ingest_v2::state::WeakIngesterState; + +#[derive(Debug, Clone, Default, Serialize, Deserialize)] +pub struct IngesterCapacityScore { + pub capacity_score: usize, + pub open_shard_count: usize, +} + +/// Periodically snapshots the ingester's WAL memory usage and open shard counts, computes +/// a capacity score, and broadcasts it to other nodes via Chitchat. +pub struct BroadcastIngesterCapacityScoreTask { + cluster: Cluster, + weak_state: WeakIngesterState, +} + +impl BroadcastIngesterCapacityScoreTask { + pub fn spawn(cluster: Cluster, weak_state: WeakIngesterState) -> JoinHandle<()> { + let mut broadcaster = Self { + cluster, + weak_state, + }; + tokio::spawn(async move { broadcaster.run().await }) + } + + async fn snapshot(&self) -> Result> { + let state = self + .weak_state + .upgrade() + .context("ingester state has been dropped")?; + + // lock fully asserts that the ingester is ready. There's a likelihood that this task runs + // before the WAL is loaded, so we make sure that the ingester is ready just in case. + if *state.status_rx.borrow() != IngesterStatus::Ready { + return Ok(None); + } + + let mut guard = state + .lock_fully() + .await + .context("failed to acquire ingester state lock")?; + + let usage = guard.mrecordlog.resource_usage(); + let disk_used = ByteSize::b(usage.disk_used_bytes as u64); + let memory_used = ByteSize::b(usage.memory_used_bytes as u64); + let capacity_score = guard + .wal_capacity_tracker + .record_and_score(disk_used, memory_used); + let (open_shard_counts, _) = guard.get_shard_snapshot(); + + Ok(Some((capacity_score, open_shard_counts))) + } + + async fn run(&mut self) { + let mut interval = tokio::time::interval(BROADCAST_INTERVAL_PERIOD); + let mut previous_sources: BTreeSet = BTreeSet::new(); + + loop { + interval.tick().await; + + let (capacity_score, open_shard_counts) = match self.snapshot().await { + Ok(Some(snapshot)) => snapshot, + Ok(None) => continue, + Err(error) => { + info!("stopping ingester capacity broadcast: {error}"); + return; + } + }; + + previous_sources = self + .broadcast_capacity(capacity_score, &open_shard_counts, &previous_sources) + .await; + } + } + + async fn broadcast_capacity( + &self, + capacity_score: usize, + open_shard_counts: &OpenShardCounts, + previous_sources: &BTreeSet, + ) -> BTreeSet { + let mut current_sources = BTreeSet::new(); + + for (index_uid, source_id, open_shard_count) in open_shard_counts { + let source_uid = SourceUid { + index_uid: index_uid.clone(), + source_id: source_id.clone(), + }; + let key = make_key(INGESTER_CAPACITY_SCORE_PREFIX, &source_uid); + let capacity = IngesterCapacityScore { + capacity_score, + open_shard_count: *open_shard_count, + }; + let value = serde_json::to_string(&capacity) + .expect("`IngesterCapacityScore` should be JSON serializable"); + self.cluster.set_self_key_value(key, value).await; + current_sources.insert(source_uid); + } + + for removed_source in previous_sources.difference(¤t_sources) { + let key = make_key(INGESTER_CAPACITY_SCORE_PREFIX, removed_source); + self.cluster.remove_self_key(&key).await; + } + + current_sources + } +} + +#[derive(Debug, Clone)] +pub struct IngesterCapacityScoreUpdate { + pub node_id: NodeId, + pub source_uid: SourceUid, + pub capacity_score: usize, + pub open_shard_count: usize, +} + +impl Event for IngesterCapacityScoreUpdate {} + +pub async fn setup_ingester_capacity_update_listener( + cluster: Cluster, + event_broker: EventBroker, +) -> ListenerHandle { + cluster + .subscribe(INGESTER_CAPACITY_SCORE_PREFIX, move |event| { + let Some(source_uid) = parse_key(event.key) else { + warn!("failed to parse source UID from key `{}`", event.key); + return; + }; + let Ok(ingester_capacity) = serde_json::from_str::(event.value) + else { + warn!("failed to parse ingester capacity `{}`", event.value); + return; + }; + let node_id: NodeId = event.node.node_id.clone().into(); + event_broker.publish(IngesterCapacityScoreUpdate { + node_id, + source_uid, + capacity_score: ingester_capacity.capacity_score, + open_shard_count: ingester_capacity.open_shard_count, + }); + }) + .await +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + use std::sync::atomic::{AtomicUsize, Ordering}; + + use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; + use quickwit_proto::types::{IndexUid, ShardId, SourceId}; + + use super::*; + use crate::ingest_v2::models::IngesterShard; + use crate::ingest_v2::state::IngesterState; + + #[tokio::test] + async fn test_snapshot_state_dropped() { + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + .await + .unwrap(); + let (_temp_dir, state) = IngesterState::for_test(cluster.clone()).await; + let weak_state = state.weak(); + drop(state); + + let task = BroadcastIngesterCapacityScoreTask { + cluster, + weak_state, + }; + assert!(task.snapshot().await.is_err()); + } + + #[tokio::test] + async fn test_broadcast_ingester_capacity() { + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + .await + .unwrap(); + let event_broker = EventBroker::default(); + + // Use 1000 bytes disk capacity so 500 used => 50% remaining, 0 delta => score = 6 + let (_temp_dir, state) = + IngesterState::for_test_with_disk_capacity(cluster.clone(), ByteSize::b(1000)).await; + let index_uid = IndexUid::for_test("test-index", 0); + let mut state_guard = state.lock_partially().await.unwrap(); + let shard = IngesterShard::new_solo( + index_uid.clone(), + SourceId::from("test-source"), + ShardId::from(0), + ) + .advertisable() + .build(); + state_guard.shards.insert(shard.queue_id(), shard); + let (open_shard_counts, _) = state_guard.get_shard_snapshot(); + let capacity_score = state_guard + .wal_capacity_tracker + .record_and_score(ByteSize::b(500), ByteSize::b(0)); + drop(state_guard); + + assert_eq!(capacity_score, 6); + + let task = BroadcastIngesterCapacityScoreTask { + cluster: cluster.clone(), + weak_state: state.weak(), + }; + + let update_counter = Arc::new(AtomicUsize::new(0)); + let update_counter_clone = update_counter.clone(); + let index_uid_clone = index_uid.clone(); + let _sub = event_broker.subscribe(move |event: IngesterCapacityScoreUpdate| { + update_counter_clone.fetch_add(1, Ordering::Release); + assert_eq!(event.source_uid.index_uid, index_uid_clone); + assert_eq!(event.source_uid.source_id, "test-source"); + assert_eq!(event.capacity_score, 6); + assert_eq!(event.open_shard_count, 1); + }); + + let _listener = + setup_ingester_capacity_update_listener(cluster.clone(), event_broker).await; + + let previous_sources = BTreeSet::new(); + task.broadcast_capacity(capacity_score, &open_shard_counts, &previous_sources) + .await; + tokio::time::sleep(BROADCAST_INTERVAL_PERIOD * 2).await; + + assert_eq!(update_counter.load(Ordering::Acquire), 1); + + let source_uid = SourceUid { + index_uid: index_uid.clone(), + source_id: SourceId::from("test-source"), + }; + let key = make_key(INGESTER_CAPACITY_SCORE_PREFIX, &source_uid); + let value = cluster.get_self_key_value(&key).await.unwrap(); + let deserialized: IngesterCapacityScore = serde_json::from_str(&value).unwrap(); + assert_eq!(deserialized.capacity_score, 6); + assert_eq!(deserialized.open_shard_count, 1); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs similarity index 91% rename from quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs rename to quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs index 98be3df36a1..6531c893c9f 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/local_shards.rs @@ -18,6 +18,7 @@ use std::time::Duration; use bytesize::ByteSize; use quickwit_cluster::{Cluster, ListenerHandle}; use quickwit_common::pubsub::{Event, EventBroker}; +use quickwit_common::ring_buffer::RingBuffer; use quickwit_common::shared_consts::INGESTER_PRIMARY_SHARDS_PREFIX; use quickwit_common::sorted_iter::{KeyDiff, SortedByKeyIterator}; use quickwit_common::tower::{ConstantRate, Rate}; @@ -27,15 +28,10 @@ use serde::{Deserialize, Serialize, Serializer}; use tokio::task::JoinHandle; use tracing::{debug, warn}; -use super::metrics::INGEST_V2_METRICS; -use super::state::WeakIngesterState; +use super::{BROADCAST_INTERVAL_PERIOD, make_key, parse_key}; use crate::RateMibPerSec; - -const BROADCAST_INTERVAL_PERIOD: Duration = if cfg!(test) { - Duration::from_millis(50) -} else { - Duration::from_secs(5) -}; +use crate::ingest_v2::metrics::INGEST_V2_METRICS; +use crate::ingest_v2::state::WeakIngesterState; const ONE_MIB: ByteSize = ByteSize::mib(1); @@ -152,7 +148,7 @@ impl LocalShardsSnapshot { /// Takes a snapshot of the primary shards hosted by the ingester at regular intervals and /// broadcasts it to other nodes via Chitchat. -pub(super) struct BroadcastLocalShardsTask { +pub struct BroadcastLocalShardsTask { cluster: Cluster, weak_state: WeakIngesterState, shard_throughput_time_series_map: ShardThroughputTimeSeriesMap, @@ -229,36 +225,24 @@ impl ShardThroughputTimeSeriesMap { #[derive(Default)] struct ShardThroughputTimeSeries { shard_state: ShardState, - measurements: [ByteSize; SHARD_THROUGHPUT_LONG_TERM_WINDOW_LEN], - len: usize, + throughput: RingBuffer, } impl ShardThroughputTimeSeries { fn last(&self) -> ByteSize { - self.measurements.last().copied().unwrap_or_default() + self.throughput.last().unwrap_or_default() } fn average(&self) -> ByteSize { - if self.len == 0 { + if self.throughput.is_empty() { return ByteSize::default(); } - let sum = self - .measurements - .iter() - .rev() - .take(self.len) - .map(ByteSize::as_u64) - .sum::(); - ByteSize::b(sum / self.len as u64) + let sum = self.throughput.iter().map(ByteSize::as_u64).sum::(); + ByteSize::b(sum / self.throughput.len() as u64) } fn record(&mut self, new_throughput_measurement: ByteSize) { - self.len = (self.len + 1).min(SHARD_THROUGHPUT_LONG_TERM_WINDOW_LEN); - self.measurements.rotate_left(1); - let Some(last_measurement) = self.measurements.last_mut() else { - return; - }; - *last_measurement = new_throughput_measurement; + self.throughput.push_back(new_throughput_measurement); } } @@ -338,13 +322,13 @@ impl BroadcastLocalShardsTask { source_uid, shard_infos, } => { - let key = make_key(source_uid); + let key = make_key(INGESTER_PRIMARY_SHARDS_PREFIX, source_uid); let value = serde_json::to_string(&shard_infos) .expect("`ShardInfos` should be JSON serializable"); self.cluster.set_self_key_value(key, value).await; } ShardInfosChange::Removed { source_uid } => { - let key = make_key(source_uid); + let key = make_key(INGESTER_PRIMARY_SHARDS_PREFIX, source_uid); self.cluster.remove_self_key(&key).await; } } @@ -371,22 +355,6 @@ impl BroadcastLocalShardsTask { } } -fn make_key(source_uid: &SourceUid) -> String { - format!( - "{INGESTER_PRIMARY_SHARDS_PREFIX}{}:{}", - source_uid.index_uid, source_uid.source_id - ) -} - -fn parse_key(key: &str) -> Option { - let (index_uid_str, source_id_str) = key.rsplit_once(':')?; - - Some(SourceUid { - index_uid: index_uid_str.parse().ok()?, - source_id: source_id_str.to_string(), - }) -} - #[derive(Debug, Clone)] pub struct LocalShardsUpdate { pub leader_id: NodeId, @@ -429,10 +397,12 @@ mod tests { use std::sync::atomic::{AtomicUsize, Ordering}; use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; + use quickwit_common::shared_consts::INGESTER_PRIMARY_SHARDS_PREFIX; use quickwit_proto::ingest::ShardState; - use quickwit_proto::types::{IndexUid, SourceId}; + use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceId, SourceUid}; use super::*; + use crate::RateMibPerSec; use crate::ingest_v2::models::IngesterShard; use crate::ingest_v2::state::IngesterState; @@ -626,30 +596,6 @@ mod tests { assert!(value_opt.is_none()); } - #[test] - fn test_make_key() { - let source_uid = SourceUid { - index_uid: IndexUid::for_test("test-index", 0), - source_id: SourceId::from("test-source"), - }; - let key = make_key(&source_uid); - assert_eq!( - key, - "ingester.primary_shards:test-index:00000000000000000000000000:test-source" - ); - } - - #[test] - fn test_parse_key() { - let key = "test-index:00000000000000000000000000:test-source"; - let source_uid = parse_key(key).unwrap(); - assert_eq!( - &source_uid.index_uid.to_string(), - "test-index:00000000000000000000000000" - ); - assert_eq!(source_uid.source_id, "test-source".to_string()); - } - #[tokio::test] async fn test_local_shards_update_listener() { let transport = ChannelTransport::default(); @@ -686,7 +632,7 @@ mod tests { index_uid: index_uid.clone(), source_id: SourceId::from("test-source"), }; - let key = make_key(&source_uid); + let key = make_key(INGESTER_PRIMARY_SHARDS_PREFIX, &source_uid); let value = serde_json::to_string(&vec![ShardInfo { shard_id: ShardId::from(1), shard_state: ShardState::Open, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/mod.rs new file mode 100644 index 00000000000..b579382af78 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast/mod.rs @@ -0,0 +1,80 @@ +// 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. + +#[allow(dead_code)] +mod capacity_score; +mod local_shards; + +use std::time::Duration; + +use quickwit_proto::types::SourceUid; + +pub(in crate::ingest_v2) const BROADCAST_INTERVAL_PERIOD: Duration = if cfg!(test) { + Duration::from_millis(50) +} else { + Duration::from_secs(5) +}; + +pub use capacity_score::{ + BroadcastIngesterCapacityScoreTask, IngesterCapacityScoreUpdate, + setup_ingester_capacity_update_listener, +}; +pub use local_shards::{ + BroadcastLocalShardsTask, LocalShardsUpdate, ShardInfo, ShardInfos, + setup_local_shards_update_listener, +}; + +fn make_key(prefix: &str, source_uid: &SourceUid) -> String { + format!("{prefix}{}:{}", source_uid.index_uid, source_uid.source_id) +} + +fn parse_key(key: &str) -> Option { + let (index_uid_str, source_id_str) = key.rsplit_once(':')?; + Some(SourceUid { + index_uid: index_uid_str.parse().ok()?, + source_id: source_id_str.to_string(), + }) +} + +#[cfg(test)] +mod tests { + use quickwit_common::shared_consts::INGESTER_PRIMARY_SHARDS_PREFIX; + use quickwit_proto::types::{IndexUid, SourceId, SourceUid}; + + use super::*; + + #[test] + fn test_make_key() { + let source_uid = SourceUid { + index_uid: IndexUid::for_test("test-index", 0), + source_id: SourceId::from("test-source"), + }; + let key = make_key(INGESTER_PRIMARY_SHARDS_PREFIX, &source_uid); + assert_eq!( + key, + "ingester.primary_shards:test-index:00000000000000000000000000:test-source" + ); + } + + #[test] + fn test_parse_key() { + let key = "test-index:00000000000000000000000000:test-source"; + let source_uid = parse_key(key).unwrap(); + assert_eq!( + &source_uid.index_uid.to_string(), + "test-index:00000000000000000000000000" + ); + assert_eq!(source_uid.source_id, "test-source".to_string()); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index e20ad123a2e..73c1fb2858d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -627,15 +627,12 @@ pub(super) mod tests { use bytes::Bytes; use quickwit_proto::ingest::ShardState; - use quickwit_proto::ingest::ingester::{ - IngesterServiceClient, IngesterStatus, MockIngesterService, - }; + use quickwit_proto::ingest::ingester::{IngesterServiceClient, MockIngesterService}; use quickwit_proto::types::queue_id; use tokio::time::timeout; use super::*; - use crate::MRecord; - use crate::ingest_v2::IngesterPoolEntry; + use crate::{IngesterPoolEntry, MRecord}; pub fn into_fetch_payload(fetch_message: FetchMessage) -> FetchPayload { match fetch_message.message.unwrap() { @@ -1332,11 +1329,8 @@ pub(super) mod tests { Ok(service_stream_1) }); - let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); - let ingester_1 = IngesterPoolEntry { - client: client_1, - status: IngesterStatus::Ready, - }; + let ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); ingester_pool.insert("test-ingester-1".into(), ingester_1); let fetch_payload = FetchPayload { @@ -1435,11 +1429,8 @@ pub(super) mod tests { "open fetch stream error".to_string(), )) }); - let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); - let ingester_0 = IngesterPoolEntry { - client: client_0, - status: IngesterStatus::Ready, - }; + let ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); let mut mock_ingester_1 = MockIngesterService::new(); let index_uid_clone = index_uid.clone(); @@ -1454,11 +1445,8 @@ pub(super) mod tests { Ok(service_stream_1) }); - let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); - let ingester_1 = IngesterPoolEntry { - client: client_1, - status: IngesterStatus::Ready, - }; + let ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); @@ -1558,11 +1546,8 @@ pub(super) mod tests { Ok(service_stream_0) }); - let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); - let ingester_0 = IngesterPoolEntry { - client: client_0, - status: IngesterStatus::Ready, - }; + let ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); let mut mock_ingester_1 = MockIngesterService::new(); let index_uid_clone = index_uid.clone(); @@ -1577,11 +1562,8 @@ pub(super) mod tests { Ok(service_stream_1) }); - let client_1 = IngesterServiceClient::from_mock(mock_ingester_1); - let ingester_1 = IngesterPoolEntry { - client: client_1, - status: IngesterStatus::Ready, - }; + let ingester_1 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_1)); ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); @@ -1684,11 +1666,8 @@ pub(super) mod tests { shard_id: ShardId::from(1), }) }); - let client_0 = IngesterServiceClient::from_mock(mock_ingester_0); - let ingester_0 = IngesterPoolEntry { - client: client_0, - status: IngesterStatus::Ready, - }; + let ingester_0 = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester_0)); ingester_pool.insert("test-ingester-0".into(), ingester_0); fault_tolerant_fetch_stream( @@ -1776,11 +1755,9 @@ pub(super) mod tests { Ok(service_stream_2) }); - let client = IngesterServiceClient::from_mock(mock_ingester); - let ingester = IngesterPoolEntry { - client, - status: IngesterStatus::Ready, - }; + let ingester = + IngesterPoolEntry::ready_with_client(IngesterServiceClient::from_mock(mock_ingester)); + ingester_pool.insert("test-ingester".into(), ingester); let fetch_payload = FetchPayload { diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index 3258a528fbd..5091784b885 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -35,16 +35,7 @@ use quickwit_proto::control_plane::{ AdviseResetShardsRequest, ControlPlaneService, ControlPlaneServiceClient, }; use quickwit_proto::indexing::ShardPositionsUpdate; -use quickwit_proto::ingest::ingester::{ - AckReplicationMessage, CloseShardsRequest, CloseShardsResponse, DecommissionRequest, - 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::ingester::*; use quickwit_proto::ingest::{ CommitTypeV2, DocBatchV2, IngestV2Error, IngestV2Result, ParseFailure, Shard, ShardIds, }; @@ -57,7 +48,7 @@ use tokio::time::{sleep, timeout}; use tracing::{debug, error, info, warn}; use super::IngesterPool; -use super::broadcast::BroadcastLocalShardsTask; +use super::broadcast::{BroadcastIngesterCapacityScoreTask, BroadcastLocalShardsTask}; use super::doc_mapper::validate_doc_batch; use super::fetch::FetchStreamTask; use super::idle::CloseIdleShardsTask; @@ -139,10 +130,18 @@ impl Ingester { idle_shard_timeout: Duration, ) -> IngestV2Result { let self_node_id: NodeId = cluster.self_node_id().into(); - let state = IngesterState::load(cluster.clone(), wal_dir_path, rate_limiter_settings).await; + let state = IngesterState::load( + cluster.clone(), + wal_dir_path, + disk_capacity, + memory_capacity, + rate_limiter_settings, + ) + .await; let weak_state = state.weak(); - BroadcastLocalShardsTask::spawn(cluster, weak_state.clone()); + BroadcastLocalShardsTask::spawn(cluster.clone(), weak_state.clone()); + BroadcastIngesterCapacityScoreTask::spawn(cluster, weak_state.clone()); CloseIdleShardsTask::spawn(weak_state, idle_shard_timeout); let ingester = Self { @@ -459,11 +458,24 @@ impl Ingester { 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); + 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, + reason: PersistFailureReason::NodeUnavailable as i32, + }; + persist_failures.push(persist_failure); + } + let persist_response = PersistResponse { + leader_id: persist_request.leader_id, + successes: Vec::new(), + failures: persist_failures, + routing_update: None, + }; + return Ok(persist_response); } // first verify if we would locally accept each subrequest { @@ -483,8 +495,7 @@ impl Ingester { subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - reason: PersistFailureReason::ShardNotFound as i32, + reason: PersistFailureReason::NoShardsAvailable as i32, }; persist_failures.push(persist_failure); continue; @@ -524,7 +535,6 @@ impl Ingester { subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, - shard_id: Some(shard_id), reason: PersistFailureReason::WalFull as i32, }; persist_failures.push(persist_failure); @@ -542,8 +552,7 @@ impl Ingester { subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, - shard_id: Some(shard_id), - reason: PersistFailureReason::ShardRateLimited as i32, + reason: PersistFailureReason::NoShardsAvailable as i32, }; persist_failures.push(persist_failure); continue; @@ -658,7 +667,7 @@ impl Ingester { // TODO: Handle replication error: // 1. Close and evict all the shards hosted by the follower. // 2. Close and evict the replication client. - // 3. Return `PersistFailureReason::ShardClosed` to router. + // 3. Return `PersistFailureReason::NodeUnavailable` to router. continue; } }; @@ -674,19 +683,12 @@ impl Ingester { for replicate_failure in replicate_response.failures { // TODO: If the replica shard is closed, close the primary shard if it is not // already. - let persist_failure_reason = match replicate_failure.reason() { - ReplicateFailureReason::Unspecified => PersistFailureReason::Unspecified, - ReplicateFailureReason::ShardNotFound => { - PersistFailureReason::ShardNotFound - } - ReplicateFailureReason::ShardClosed => PersistFailureReason::ShardClosed, - ReplicateFailureReason::WalFull => PersistFailureReason::WalFull, - }; + let persist_failure_reason: PersistFailureReason = + replicate_failure.reason().into(); let persist_failure = PersistFailure { subrequest_id: replicate_failure.subrequest_id, index_uid: replicate_failure.index_uid, source_id: replicate_failure.source_id, - shard_id: replicate_failure.shard_id, reason: persist_failure_reason as i32, }; persist_failures.push(persist_failure); @@ -721,7 +723,7 @@ impl Ingester { "failed to persist records to shard `{queue_id}`: {io_error}" ); shards_to_close.insert(queue_id); - PersistFailureReason::ShardClosed + PersistFailureReason::NodeUnavailable } AppendDocBatchError::QueueNotFound(_) => { error!( @@ -729,14 +731,13 @@ impl Ingester { not found" ); shards_to_delete.insert(queue_id); - PersistFailureReason::ShardNotFound + PersistFailureReason::NodeUnavailable } }; 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: reason as i32, }; persist_failures.push(persist_failure); @@ -788,15 +789,35 @@ impl Ingester { } } let wal_usage = state_guard.mrecordlog.resource_usage(); - drop(state_guard); - let disk_used = wal_usage.disk_used_bytes as u64; + let memory_used = wal_usage.memory_used_bytes as u64; + let (open_shard_counts, closed_shards) = state_guard.get_shard_snapshot(); + let capacity_score = state_guard + .wal_capacity_tracker + .score(ByteSize::b(disk_used), ByteSize::b(memory_used)) + as u32; + drop(state_guard); if disk_used >= self.disk_capacity.as_u64() * 90 / 100 { self.background_reset_shards(); } report_wal_usage(wal_usage); + let source_shard_updates = open_shard_counts + .into_iter() + .map(|(index_uid, source_id, count)| SourceShardUpdate { + index_uid: Some(index_uid), + source_id, + open_shard_count: count as u32, + }) + .collect(); + + let routing_update = RoutingUpdate { + capacity_score, + source_shard_updates, + closed_shards, + }; + #[cfg(test)] { persist_successes.sort_by_key(|success| success.subrequest_id); @@ -807,6 +828,7 @@ impl Ingester { leader_id, successes: persist_successes, failures: persist_failures, + routing_update: Some(routing_update), }; Ok(persist_response) } @@ -1294,12 +1316,12 @@ mod tests { use tonic::transport::{Endpoint, Server}; 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}; + use crate::{IngesterPoolEntry, MRecord}; const MAX_GRPC_MESSAGE_SIZE: ByteSize = ByteSize::mib(1); @@ -1746,14 +1768,12 @@ mod tests { subrequest_id: 0, 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_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"}"#, r#"{"doc": "test-doc-111"}"#, @@ -1770,7 +1790,6 @@ mod tests { assert_eq!(persist_success_0.subrequest_id, 0); 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!( persist_success_0.replication_position_inclusive, Some(Position::offset(1u64)) @@ -1780,7 +1799,6 @@ mod tests { assert_eq!(persist_success_1.subrequest_id, 1); 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!( persist_success_1.replication_position_inclusive, Some(Position::offset(2u64)) @@ -1867,8 +1885,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(0)), + source_id: "test-source".to_string(), doc_batch: None, }], }; @@ -1881,7 +1898,6 @@ mod tests { assert_eq!(persist_success.subrequest_id, 0); assert_eq!(persist_success.index_uid(), &index_uid); assert_eq!(persist_success.source_id, "test-source"); - assert_eq!(persist_success.shard_id(), ShardId::from(0)); assert_eq!( persist_success.replication_position_inclusive, Some(Position::Beginning) @@ -1929,8 +1945,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(0)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([ "", // invalid "[]", // invalid @@ -2005,8 +2020,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(0)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([ "", // invalid "[]", // invalid @@ -2069,8 +2083,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(0)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["", "[]", r#"{"foo": "bar"}"#])), }], }; @@ -2131,8 +2144,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(0)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test(["", "[]", r#"{"foo": "bar"}"#])), }], }; @@ -2144,7 +2156,7 @@ mod tests { let persist_failure = &persist_response.failures[0]; assert_eq!( persist_failure.reason(), - PersistFailureReason::ShardRateLimited + PersistFailureReason::NoShardsAvailable ); } @@ -2189,8 +2201,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: source_id, - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"test-doc-foo"#])), }], }; @@ -2203,8 +2214,10 @@ mod tests { assert_eq!(persist_failure.subrequest_id, 0); assert_eq!(persist_failure.index_uid(), &index_uid); assert_eq!(persist_failure.source_id, "test-source"); - assert_eq!(persist_failure.shard_id(), ShardId::from(1)); - assert_eq!(persist_failure.reason(), PersistFailureReason::ShardClosed,); + assert_eq!( + persist_failure.reason(), + PersistFailureReason::NodeUnavailable, + ); let state_guard = ingester.state.lock_fully().await.unwrap(); let shard = state_guard.shards.get(&queue_id).unwrap(); @@ -2237,8 +2250,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-foo"}"#])), }], }; @@ -2251,10 +2263,9 @@ mod tests { assert_eq!(persist_failure.subrequest_id, 0); assert_eq!(persist_failure.index_uid(), &index_uid); assert_eq!(persist_failure.source_id, "test-source"); - assert_eq!(persist_failure.shard_id(), ShardId::from(1)); assert_eq!( persist_failure.reason(), - PersistFailureReason::ShardNotFound + PersistFailureReason::NodeUnavailable ); let state_guard = ingester.state.lock_fully().await.unwrap(); @@ -2276,13 +2287,15 @@ mod tests { .build() .await; - leader_ctx.ingester_pool.insert( - follower_ctx.node_id.clone(), - IngesterPoolEntry { - client: IngesterServiceClient::new(follower.clone()), - status: IngesterStatus::Ready, - }, - ); + let ingester_pool_entry = IngesterPoolEntry { + client: IngesterServiceClient::new(follower.clone()), + status: IngesterStatus::Ready, + availability_zone: None, + }; + + leader_ctx + .ingester_pool + .insert(follower_ctx.node_id.clone(), ingester_pool_entry); let index_uid = IndexUid::for_test("test-index", 0); let index_uid2 = IndexUid::for_test("test-index", 1); @@ -2337,15 +2350,13 @@ mod tests { PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([ r#"{"doc": "test-doc-110"}"#, r#"{"doc": "test-doc-111"}"#, @@ -2488,13 +2499,16 @@ mod tests { MAX_GRPC_MESSAGE_SIZE, None, ); - let follower_ingester = IngesterPoolEntry { + + let ingester_pool_entry = IngesterPoolEntry { client: follower_client, status: IngesterStatus::Ready, + availability_zone: None, }; + leader_ctx .ingester_pool - .insert(follower_ctx.node_id.clone(), follower_ingester); + .insert(follower_ctx.node_id.clone(), ingester_pool_entry); let index_uid = IndexUid::for_test("test-index", 0); let index_uid2 = IndexUid::for_test("test-index", 1); @@ -2549,15 +2563,13 @@ mod tests { PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }, PersistSubrequest { subrequest_id: 1, index_uid: Some(index_uid2.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([ r#"{"doc": "test-doc-110"}"#, r#"{"doc": "test-doc-111"}"#, @@ -2673,8 +2685,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id, - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; @@ -2687,10 +2698,9 @@ mod tests { assert_eq!(persist_failure.subrequest_id, 0); assert_eq!(persist_failure.index_uid(), &index_uid); assert_eq!(persist_failure.source_id, "test-source"); - assert_eq!(persist_failure.shard_id(), ShardId::from(1)); assert_eq!( persist_failure.reason(), - PersistFailureReason::ShardNotFound + PersistFailureReason::NoShardsAvailable ); let state_guard = ingester.state.lock_fully().await.unwrap(); @@ -2753,8 +2763,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; @@ -2767,10 +2776,9 @@ mod tests { assert_eq!(persist_failure.subrequest_id, 0); assert_eq!(persist_failure.index_uid(), &index_uid); assert_eq!(persist_failure.source_id, "test-source"); - assert_eq!(persist_failure.shard_id(), ShardId::from(1)); assert_eq!( persist_failure.reason(), - PersistFailureReason::ShardRateLimited + PersistFailureReason::NoShardsAvailable ); let state_guard = ingester.state.lock_fully().await.unwrap(); @@ -2835,8 +2843,7 @@ mod tests { subrequests: vec![PersistSubrequest { subrequest_id: 0, index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), + source_id: "test-source".to_string(), doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), }], }; @@ -2849,7 +2856,6 @@ mod tests { assert_eq!(persist_failure.subrequest_id, 0); assert_eq!(persist_failure.index_uid(), &index_uid); assert_eq!(persist_failure.source_id, "test-source"); - assert_eq!(persist_failure.shard_id(), ShardId::from(1)); assert_eq!(persist_failure.reason(), PersistFailureReason::WalFull); let state_guard = ingester.state.lock_fully().await.unwrap(); @@ -2866,6 +2872,98 @@ mod tests { .assert_records_eq(&queue_id_01, .., &[]); } + #[tokio::test] + async fn test_ingester_persist_returns_routing_update() { + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; + + let index_uid_0 = IndexUid::for_test("test-index-0", 0); + let index_uid_1 = 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!( + r#"{{ + "doc_mapping_uid": "{doc_mapping_uid}" + }}"# + ); + let init_shards_request = InitShardsRequest { + subrequests: vec![ + InitShardSubrequest { + subrequest_id: 0, + shard: Some(Shard { + 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(), + doc_mapping_uid: Some(doc_mapping_uid), + ..Default::default() + }), + doc_mapping_json: doc_mapping_json.clone(), + validate_docs: false, + }, + InitShardSubrequest { + subrequest_id: 1, + shard: Some(Shard { + 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(), + doc_mapping_uid: Some(doc_mapping_uid), + ..Default::default() + }), + doc_mapping_json, + validate_docs: false, + }, + ], + }; + ingester.init_shards(init_shards_request).await.unwrap(); + + let persist_request = PersistRequest { + leader_id: ingester_ctx.node_id.to_string(), + commit_type: CommitTypeV2::Force as i32, + subrequests: vec![ + PersistSubrequest { + subrequest_id: 0, + index_uid: Some(index_uid_0.clone()), + source_id: source_id.clone(), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-010"}"#])), + }, + PersistSubrequest { + subrequest_id: 1, + index_uid: Some(index_uid_1.clone()), + source_id: source_id.clone(), + doc_batch: Some(DocBatchV2::for_test([r#"{"doc": "test-doc-110"}"#])), + }, + ], + }; + let persist_response = ingester.persist(persist_request).await.unwrap(); + assert_eq!(persist_response.successes.len(), 2); + + let routing_update = persist_response + .routing_update + .expect("routing update should be present"); + + assert!( + routing_update.capacity_score > 0, + "capacity score should be non-zero after a small persist" + ); + + let mut source_shard_updates = routing_update.source_shard_updates; + source_shard_updates.sort_by(|a, b| a.index_uid().cmp(b.index_uid())); + + assert_eq!(source_shard_updates.len(), 2); + assert_eq!(source_shard_updates[0].index_uid(), &index_uid_0); + assert_eq!(source_shard_updates[0].source_id, source_id.as_str()); + assert_eq!(source_shard_updates[0].open_shard_count, 1); + assert_eq!(source_shard_updates[1].index_uid(), &index_uid_1); + assert_eq!(source_shard_updates[1].source_id, source_id.as_str()); + assert_eq!(source_shard_updates[1].open_shard_count, 1); + + assert!(routing_update.closed_shards.is_empty()); + } + #[tokio::test] async fn test_ingester_open_replication_stream() { let (_ingester_ctx, ingester) = IngesterForTest::default() diff --git a/quickwit/quickwit-ingest/src/ingest_v2/metrics.rs b/quickwit/quickwit-ingest/src/ingest_v2/metrics.rs index 1fb32c0b2fd..87975a3c462 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/metrics.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/metrics.rs @@ -82,12 +82,20 @@ pub(super) struct IngestV2Metrics { pub wal_disk_used_bytes: IntGauge, pub wal_memory_used_bytes: IntGauge, pub ingest_results: IngestResultMetrics, + pub ingest_attempts: IntCounterVec<1>, } impl Default for IngestV2Metrics { fn default() -> Self { Self { ingest_results: IngestResultMetrics::default(), + ingest_attempts: new_counter_vec::<1>( + "ingest_attempts", + "Number of routing attempts by AZ locality", + "ingest", + &[], + ["az_routing"], + ), reset_shards_operations_total: new_counter_vec( "reset_shards_operations_total", "Total number of reset shards operations performed.", diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index 977b50581b7..a2b1095bdf3 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -29,6 +29,7 @@ mod replication; mod router; mod routing_table; mod state; +mod wal_capacity_tracker; mod workbench; use std::collections::HashMap; @@ -37,7 +38,10 @@ use std::ops::{Add, AddAssign}; use std::time::Duration; use std::{env, fmt}; -pub use broadcast::{LocalShardsUpdate, ShardInfo, ShardInfos, setup_local_shards_update_listener}; +pub use broadcast::{ + LocalShardsUpdate, ShardInfo, ShardInfos, setup_ingester_capacity_update_listener, + setup_local_shards_update_listener, +}; use bytes::buf::Writer; use bytes::{BufMut, BytesMut}; use bytesize::ByteSize; @@ -45,7 +49,9 @@ use quickwit_common::tower::Pool; 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}; +use quickwit_proto::types::{ + DocUid, DocUidGenerator, IndexId, IndexUid, NodeId, SourceId, SubrequestId, +}; use serde::Serialize; use tracing::{error, info}; use workbench::pending_subrequests; @@ -59,10 +65,32 @@ use self::mrecord::MRECORD_HEADER_LEN; pub use self::mrecord::{MRecord, decoded_mrecords}; pub use self::router::IngestRouter; -#[derive(Clone)] +/// An ingester as represented in the pool, bundling the gRPC client with node metadata. +#[derive(Debug, Clone)] pub struct IngesterPoolEntry { pub client: IngesterServiceClient, pub status: IngesterStatus, + pub availability_zone: Option, +} + +impl IngesterPoolEntry { + #[cfg(any(test, feature = "testsuite"))] + pub fn ready_with_client(client: IngesterServiceClient) -> Self { + IngesterPoolEntry { + client, + status: IngesterStatus::Ready, + availability_zone: None, + } + } + + #[cfg(any(test, feature = "testsuite"))] + pub fn mocked_ingester() -> Self { + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + availability_zone: None, + } + } } pub type IngesterPool = Pool; @@ -74,6 +102,8 @@ pub type LeaderId = NodeId; pub type FollowerId = NodeId; +pub type OpenShardCounts = Vec<(IndexUid, SourceId, usize)>; + const IDLE_SHARD_TIMEOUT_ENV_KEY: &str = "QW_IDLE_SHARD_TIMEOUT_SECS"; const DEFAULT_IDLE_SHARD_TIMEOUT: Duration = Duration::from_secs(15 * 60); // 15 minutes diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 12a031960ab..581b648ae71 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -27,32 +27,30 @@ use quickwit_proto::control_plane::{ ControlPlaneService, ControlPlaneServiceClient, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsSubrequest, }; -use quickwit_proto::indexing::ShardPositionsUpdate; use quickwit_proto::ingest::ingester::{ IngesterService, PersistFailureReason, PersistRequest, PersistResponse, PersistSubrequest, }; use quickwit_proto::ingest::router::{ IngestFailureReason, IngestRequestV2, IngestResponseV2, IngestRouterService, }; -use quickwit_proto::ingest::{ - CommitTypeV2, IngestV2Error, IngestV2Result, RateLimitingCause, ShardState, -}; -use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceId, SubrequestId}; +use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, RateLimitingCause}; +use quickwit_proto::types::{NodeId, SubrequestId}; use serde_json::{Value as JsonValue, json}; use tokio::sync::{Mutex, Semaphore}; use tokio::time::error::Elapsed; use tracing::{error, info}; -use super::broadcast::LocalShardsUpdate; +use super::broadcast::IngesterCapacityScoreUpdate; use super::debouncing::{ DebouncedGetOrCreateOpenShardsRequest, GetOrCreateOpenShardsRequestDebouncer, }; use super::ingester::PERSIST_REQUEST_TIMEOUT; use super::metrics::IngestResultMetrics; -use super::routing_table::{NextOpenShardError, RoutingTable}; +use super::routing_table::RoutingTable; use super::workbench::IngestWorkbench; use super::{IngesterPool, pending_subrequests}; -use crate::{LeaderId, get_ingest_router_buffer_size}; +use crate::get_ingest_router_buffer_size; +use crate::ingest_v2::metrics::INGEST_V2_METRICS; /// Duration after which ingest requests time out with [`IngestV2Error::Timeout`]. fn ingest_request_timeout() -> Duration { @@ -103,7 +101,7 @@ pub struct IngestRouter { struct RouterState { // Debounces `GetOrCreateOpenShardsRequest` requests to the control plane. debouncer: GetOrCreateOpenShardsRequestDebouncer, - // Holds the routing table mapping index and source IDs to shards. + // Routing table of nodes, their WAL capacity, and the number of open shards per source. routing_table: RoutingTable, } @@ -123,13 +121,11 @@ impl IngestRouter { ingester_pool: IngesterPool, replication_factor: usize, event_broker: EventBroker, + self_availability_zone: Option, ) -> Self { let state = Arc::new(Mutex::new(RouterState { debouncer: GetOrCreateOpenShardsRequestDebouncer::default(), - routing_table: RoutingTable { - self_node_id: self_node_id.clone(), - table: HashMap::default(), - }, + routing_table: RoutingTable::new(self_availability_zone), })); let ingest_semaphore_permits = get_ingest_router_buffer_size().as_u64() as usize; let ingest_semaphore = Arc::new(Semaphore::new(ingest_semaphore_permits)); @@ -148,10 +144,7 @@ impl IngestRouter { pub fn subscribe(&self) { let weak_router_state = WeakRouterState(Arc::downgrade(&self.state)); self.event_broker - .subscribe::(weak_router_state.clone()) - .forever(); - self.event_broker - .subscribe::(weak_router_state) + .subscribe::(weak_router_state) .forever(); } @@ -163,22 +156,19 @@ impl IngestRouter { ingester_pool: &IngesterPool, ) -> DebouncedGetOrCreateOpenShardsRequest { let mut debounced_request = DebouncedGetOrCreateOpenShardsRequest::default(); - - // `closed_shards` and `unavailable_leaders` are populated by calls to `has_open_shards` - // as we're looking for open shards to route the subrequests to. - let unavailable_leaders: &mut HashSet = &mut workbench.unavailable_leaders; + let unavailable_leaders: &HashSet = &workbench.unavailable_leaders; let mut state_guard = self.state.lock().await; for subrequest in pending_subrequests(&workbench.subworkbenches) { - if !state_guard.routing_table.has_open_shards( + if !state_guard.routing_table.has_open_nodes( &subrequest.index_id, &subrequest.source_id, ingester_pool, - &mut debounced_request.closed_shards, unavailable_leaders, ) { - // No shard available! Let's attempt to create one. + // No known nodes with open shards for this source. Ask the control + // plane to create shards so we have somewhere to route to. let acquire_result = state_guard .debouncer .acquire(&subrequest.index_id, &subrequest.source_id); @@ -200,8 +190,11 @@ impl IngestRouter { } drop(state_guard); - if !debounced_request.is_empty() && !debounced_request.closed_shards.is_empty() { - info!(closed_shards=?debounced_request.closed_shards, "reporting closed shard(s) to control plane"); + if !debounced_request.is_empty() && !workbench.closed_shards.is_empty() { + info!(closed_shards=?workbench.closed_shards, "reporting closed shard(s) to control plane"); + debounced_request + .closed_shards + .append(&mut workbench.closed_shards); } if !debounced_request.is_empty() && !unavailable_leaders.is_empty() { info!(unavailable_leaders=?unavailable_leaders, "reporting unavailable leader(s) to control plane"); @@ -259,7 +252,7 @@ impl IngestRouter { let mut state_guard = self.state.lock().await; for success in response.successes { - state_guard.routing_table.replace_shards( + state_guard.routing_table.merge_from_shards( success.index_uid().clone(), success.source_id, success.open_shards, @@ -277,12 +270,13 @@ impl IngestRouter { workbench: &mut IngestWorkbench, mut persist_futures: FuturesUnordered>, ) { - let mut closed_shards: HashMap<(IndexUid, SourceId), Vec> = HashMap::new(); - let mut deleted_shards: HashMap<(IndexUid, SourceId), Vec> = HashMap::new(); + let mut unavailable_leaders: HashSet = HashSet::new(); while let Some((persist_summary, persist_result)) = persist_futures.next().await { match persist_result { Ok(persist_response) => { + let leader_id = NodeId::from(persist_response.leader_id.clone()); + for persist_success in persist_response.successes { workbench.record_persist_success(persist_success); } @@ -290,37 +284,37 @@ impl IngestRouter { workbench.record_persist_failure(&persist_failure); match persist_failure.reason() { - PersistFailureReason::ShardClosed => { - let shard_id = persist_failure.shard_id().clone(); - let index_uid: IndexUid = persist_failure.index_uid().clone(); - let source_id: SourceId = persist_failure.source_id; - closed_shards - .entry((index_uid, source_id)) - .or_default() - .push(shard_id); + PersistFailureReason::NoShardsAvailable => { + // For non-critical failures, we don't mark the nodes unavailable; + // a routing update is piggybacked on PersistResponses, so shard + // counts and capacity scores will be fresh on the next try. } - PersistFailureReason::ShardNotFound => { - let shard_id = persist_failure.shard_id().clone(); - let index_uid: IndexUid = persist_failure.index_uid().clone(); - let source_id: SourceId = persist_failure.source_id; - deleted_shards - .entry((index_uid, source_id)) - .or_default() - .push(shard_id); - } - PersistFailureReason::WalFull - | PersistFailureReason::ShardRateLimited => { - // Let's record that the shard is rate limited or that the ingester - // that hosts has its wal full. - // - // That way we will avoid to retry the persist request on the very - // same node. - let shard_id = persist_failure.shard_id().clone(); - workbench.rate_limited_shards.insert(shard_id); + PersistFailureReason::NodeUnavailable + | PersistFailureReason::WalFull + | PersistFailureReason::Timeout => { + unavailable_leaders.insert(leader_id.clone()); } _ => {} } } + + if let Some(routing_update) = persist_response.routing_update { + // Since we just talked to the node, we take advantage and use the + // opportunity to get a fresh routing update. + let mut state_guard = self.state.lock().await; + for shard_update in routing_update.source_shard_updates { + state_guard.routing_table.apply_capacity_update( + leader_id.clone(), + shard_update.index_uid().clone(), + shard_update.source_id, + routing_update.capacity_score as usize, + shard_update.open_shard_count as usize, + ); + } + drop(state_guard); + + workbench.closed_shards.extend(routing_update.closed_shards); + } } Err(persist_error) => { if workbench.is_last_attempt() { @@ -340,20 +334,7 @@ impl IngestRouter { } }; } - if !closed_shards.is_empty() || !deleted_shards.is_empty() { - let mut state_guard = self.state.lock().await; - - for ((index_uid, source_id), shard_ids) in closed_shards { - state_guard - .routing_table - .close_shards(&index_uid, source_id, &shard_ids); - } - for ((index_uid, source_id), shard_ids) in deleted_shards { - state_guard - .routing_table - .delete_shards(&index_uid, source_id, &shard_ids); - } - } + workbench.unavailable_leaders.extend(unavailable_leaders); } async fn batch_persist(&self, workbench: &mut IngestWorkbench, commit_type: CommitTypeV2) { @@ -365,47 +346,43 @@ impl IngestRouter { self.populate_routing_table_debounced(workbench, debounced_request) .await; - // Subrequests for which no shards are available to route the subrequests to. + let unavailable_leaders = &workbench.unavailable_leaders; let mut no_shards_available_subrequest_ids: Vec = Vec::new(); - // Subrequests for which the shards are rate limited. - let mut rate_limited_subrequest_ids: Vec = Vec::new(); - - let mut per_leader_persist_subrequests: HashMap<&LeaderId, Vec> = + let mut per_leader_persist_subrequests: HashMap<&NodeId, Vec> = HashMap::new(); - let rate_limited_shards: &HashSet = &workbench.rate_limited_shards; let state_guard = self.state.lock().await; for subrequest in pending_subrequests(&workbench.subworkbenches) { - let next_open_shard_res_opt = state_guard - .routing_table - .find_entry(&subrequest.index_id, &subrequest.source_id) - .map(|entry| { - entry.next_open_shard_round_robin(&self.ingester_pool, rate_limited_shards) - }); - let next_open_shard = match next_open_shard_res_opt { - Some(Ok(next_open_shard)) => next_open_shard, - Some(Err(NextOpenShardError::RateLimited)) => { - rate_limited_subrequest_ids.push(subrequest.subrequest_id); - continue; - } - Some(Err(NextOpenShardError::NoShardsAvailable)) | None => { + let ingester_node = state_guard.routing_table.pick_node( + &subrequest.index_id, + &subrequest.source_id, + &self.ingester_pool, + unavailable_leaders, + ); + + let ingester_node = match ingester_node { + Some(node) => node, + None => { no_shards_available_subrequest_ids.push(subrequest.subrequest_id); continue; } }; + let az_locality = state_guard + .routing_table + .classify_az_locality(&ingester_node.node_id, &self.ingester_pool); + INGEST_V2_METRICS + .ingest_attempts + .with_label_values([az_locality]) + .inc(); let persist_subrequest = PersistSubrequest { subrequest_id: subrequest.subrequest_id, - index_uid: next_open_shard.index_uid.clone().into(), - source_id: next_open_shard.source_id.clone(), - // We don't necessarily persist to this shard. We persist to the shard with the most - // capacity on that node. - // TODO: Clean this up. - shard_id: Some(next_open_shard.shard_id.clone()), + index_uid: Some(ingester_node.index_uid.clone()), + source_id: subrequest.source_id.clone(), doc_batch: subrequest.doc_batch.clone(), }; per_leader_persist_subrequests - .entry(&next_open_shard.leader_id) + .entry(&ingester_node.node_id) .or_default() .push(persist_subrequest); } @@ -417,7 +394,7 @@ impl IngestRouter { .iter() .map(|subrequest| subrequest.subrequest_id) .collect(); - let Some(ingester) = self.ingester_pool.get(&leader_id) else { + let Some(ingester) = self.ingester_pool.get(&leader_id).map(|h| h.client) else { no_shards_available_subrequest_ids.extend(subrequest_ids); continue; }; @@ -434,7 +411,7 @@ impl IngestRouter { let persist_future = async move { let persist_result = tokio::time::timeout( PERSIST_REQUEST_TIMEOUT, - ingester.client.persist(persist_request), + ingester.persist(persist_request), ) .await .unwrap_or_else(|_| { @@ -453,9 +430,6 @@ impl IngestRouter { for subrequest_id in no_shards_available_subrequest_ids { workbench.record_no_shards_available(subrequest_id); } - for subrequest_id in rate_limited_subrequest_ids { - workbench.record_rate_limited(subrequest_id); - } self.process_persist_results(workbench, persist_futures) .await; } @@ -508,7 +482,7 @@ impl IngestRouter { pub async fn debug_info(&self) -> JsonValue { let state_guard = self.state.lock().await; - let routing_table_json = state_guard.routing_table.debug_info(); + let routing_table_json = state_guard.routing_table.debug_info(&self.ingester_pool); json!({ "routing_table": routing_table_json, @@ -518,8 +492,7 @@ impl IngestRouter { fn update_ingest_metrics(ingest_result: &IngestV2Result, num_subrequests: usize) { let num_subrequests = num_subrequests as u64; - let ingest_results_metrics: &IngestResultMetrics = - &crate::ingest_v2::metrics::INGEST_V2_METRICS.ingest_results; + let ingest_results_metrics: &IngestResultMetrics = &INGEST_V2_METRICS.ingest_results; match ingest_result { Ok(ingest_response) => { ingest_results_metrics @@ -633,67 +606,22 @@ impl IngestRouterService for IngestRouter { struct WeakRouterState(Weak>); #[async_trait] -impl EventSubscriber for WeakRouterState { - async fn handle_event(&mut self, local_shards_update: LocalShardsUpdate) { +impl EventSubscriber for WeakRouterState { + async fn handle_event(&mut self, update: IngesterCapacityScoreUpdate) { let Some(state) = self.0.upgrade() else { return; }; - let leader_id = local_shards_update.leader_id; - let index_uid = local_shards_update.source_uid.index_uid; - let source_id = local_shards_update.source_uid.source_id; - - let mut open_shard_ids: Vec = Vec::new(); - let mut closed_shard_ids: Vec = Vec::new(); - - for shard_info in local_shards_update.shard_infos { - match shard_info.shard_state { - ShardState::Open => open_shard_ids.push(shard_info.shard_id), - ShardState::Closed => closed_shard_ids.push(shard_info.shard_id), - ShardState::Unavailable | ShardState::Unspecified => { - // Ingesters never broadcast the `Unavailable`` state because, from their point - // of view, they are never unavailable. - } - } - } let mut state_guard = state.lock().await; - - state_guard - .routing_table - .close_shards(&index_uid, &source_id, &closed_shard_ids); - - state_guard.routing_table.insert_open_shards( - &leader_id, - index_uid, - source_id, - &open_shard_ids, + state_guard.routing_table.apply_capacity_update( + update.node_id, + update.source_uid.index_uid, + update.source_uid.source_id, + update.capacity_score, + update.open_shard_count, ); } } -#[async_trait] -impl EventSubscriber for WeakRouterState { - async fn handle_event(&mut self, shard_positions_update: ShardPositionsUpdate) { - let Some(state) = self.0.upgrade() else { - return; - }; - let mut deleted_shard_ids: Vec = Vec::new(); - - for (shard_id, shard_position) in shard_positions_update.updated_shard_positions { - if shard_position.is_eof() { - deleted_shard_ids.push(shard_id); - } - } - let mut state_guard = state.lock().await; - - let index_uid = shard_positions_update.source_uid.index_uid; - let source_id = shard_positions_update.source_uid.source_id; - - state_guard - .routing_table - .delete_shards(&index_uid, &source_id, &deleted_shard_ids); - } -} - pub(super) struct PersistRequestSummary { pub leader_id: NodeId, pub subrequest_ids: Vec, @@ -701,29 +629,22 @@ pub(super) struct PersistRequestSummary { #[cfg(test)] mod tests { - use std::collections::BTreeSet; - - use mockall::Sequence; use quickwit_proto::control_plane::{ GetOrCreateOpenShardsFailure, GetOrCreateOpenShardsFailureReason, GetOrCreateOpenShardsResponse, GetOrCreateOpenShardsSuccess, MockControlPlaneService, }; use quickwit_proto::ingest::ingester::{ IngesterServiceClient, IngesterStatus, MockIngesterService, PersistFailure, - PersistResponse, PersistSuccess, + PersistResponse, PersistSuccess, RoutingUpdate, SourceShardUpdate, }; use quickwit_proto::ingest::router::IngestSubrequest; use quickwit_proto::ingest::{ - CommitTypeV2, DocBatchV2, ParseFailure, ParseFailureReason, Shard, ShardIds, ShardState, + CommitTypeV2, DocBatchV2, ParseFailure, ParseFailureReason, Shard, ShardState, }; - use quickwit_proto::types::{DocUid, Position, SourceUid}; - use tokio::task::yield_now; + use quickwit_proto::types::{DocUid, IndexUid, Position, ShardId, SourceUid}; 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::IngesterPoolEntry; use crate::ingest_v2::workbench::SubworkbenchFailure; #[tokio::test] @@ -739,6 +660,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let mut workbench = IngestWorkbench::default(); let (get_or_create_open_shard_request_opt, rendezvous) = router @@ -748,34 +670,16 @@ mod tests { assert!(get_or_create_open_shard_request_opt.is_none()); assert!(rendezvous.is_empty()); - let mut state_guard = router.state.lock().await; - - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - state_guard.routing_table.table.insert( - ("test-index-0".into(), "test-source".into()), - RoutingTableEntry { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - local_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - 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(), - shard_id: ShardId::from(2), - shard_state: ShardState::Open, - leader_id: "test-ingester-0".into(), - }, - ], - ..Default::default() - }, - ); - drop(state_guard); + { + let mut state_guard = router.state.lock().await; + state_guard.routing_table.apply_capacity_update( + "test-ingester-0".into(), + IndexUid::for_test("test-index-0", 0), + "test-source".to_string(), + 8, + 1, + ); + } let ingest_subrequests: Vec = vec![ IngestSubrequest { @@ -811,24 +715,12 @@ mod tests { assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(get_or_create_open_shard_request.closed_shards.len(), 1); - assert_eq!( - get_or_create_open_shard_request.closed_shards[0], - ShardIds { - index_uid: Some(IndexUid::for_test("test-index-0", 0)), - source_id: "test-source".to_string(), - shard_ids: vec![ShardId::from(1)], - } - ); - assert_eq!( - get_or_create_open_shard_request.unavailable_leaders.len(), - 1 - ); - assert_eq!( - get_or_create_open_shard_request.unavailable_leaders[0], - "test-ingester-0" + assert!( + get_or_create_open_shard_request + .unavailable_leaders + .is_empty() ); - assert_eq!(workbench.unavailable_leaders.len(), 1); + assert!(workbench.unavailable_leaders.is_empty()); let (get_or_create_open_shard_request_opt, rendezvous_2) = router .make_get_or_create_open_shard_request(&mut workbench, &ingester_pool) @@ -843,34 +735,31 @@ mod tests { drop(rendezvous_1); drop(rendezvous_2); - 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_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry::mocked_ingester(), + ); { - // Ingester-0 has been marked as unavailable due to the previous requests. + // Ingester-0 is in pool and in table, but marked unavailable on the workbench + // (simulating a prior transport error). has_open_nodes returns false → both + // subrequests trigger CP request. + workbench + .unavailable_leaders + .insert("test-ingester-0".into()); let (get_or_create_open_shard_request_opt, _rendezvous) = router .make_get_or_create_open_shard_request(&mut workbench, &ingester_pool) .await .take(); let get_or_create_open_shard_request = get_or_create_open_shard_request_opt.unwrap(); assert_eq!(get_or_create_open_shard_request.subrequests.len(), 2); - assert_eq!(workbench.unavailable_leaders.len(), 1); assert_eq!( - workbench - .unavailable_leaders - .iter() - .next() - .unwrap() - .to_string(), - "test-ingester-0" + get_or_create_open_shard_request.unavailable_leaders.len(), + 1 ); } { - // With a fresh workbench, the ingester is not marked as unavailable, and present in the - // pool. + // Fresh workbench: ingester-0 is in pool, in table, and NOT unavailable. + // has_open_nodes returns true for index-0 → only index-1 triggers request. let mut workbench = IngestWorkbench::new(ingest_subrequests, 3); let (get_or_create_open_shard_request_opt, _rendezvous) = router .make_get_or_create_open_shard_request(&mut workbench, &ingester_pool) @@ -883,9 +772,10 @@ mod tests { assert_eq!(subrequest.index_id, "test-index-1"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!( - get_or_create_open_shard_request.unavailable_leaders.len(), - 0 + assert!( + get_or_create_open_shard_request + .unavailable_leaders + .is_empty() ); } } @@ -930,6 +820,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), ..Default::default() }], }, @@ -943,6 +834,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), ..Default::default() }, Shard { @@ -950,6 +842,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), ..Default::default() }, ], @@ -981,6 +874,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![ IngestSubrequest { @@ -1040,23 +934,6 @@ mod tests { .populate_routing_table(&mut workbench, get_or_create_open_shards_request) .await; - let state_guard = router.state.lock().await; - let routing_table = &state_guard.routing_table; - assert_eq!(routing_table.len(), 2); - - let routing_entry_0 = routing_table - .find_entry("test-index-0", "test-source") - .unwrap(); - assert_eq!(routing_entry_0.len(), 1); - assert_eq!(routing_entry_0.all_shards()[0].shard_id, ShardId::from(1)); - - let routing_entry_1 = routing_table - .find_entry("test-index-1", "test-source") - .unwrap(); - assert_eq!(routing_entry_1.len(), 2); - assert_eq!(routing_entry_1.all_shards()[0].shard_id, ShardId::from(1)); - assert_eq!(routing_entry_1.all_shards()[1].shard_id, ShardId::from(2)); - let subworkbench = workbench.subworkbenches.get(&2).unwrap(); assert!(matches!( subworkbench.last_failure_opt, @@ -1096,6 +973,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1155,6 +1033,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1185,6 +1064,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1211,6 +1091,11 @@ mod tests { ..Default::default() }], failures: Vec::new(), + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), }); (persist_summary, persist_result) }); @@ -1237,6 +1122,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![IngestSubrequest { subrequest_id: 0, @@ -1260,9 +1146,13 @@ mod tests { subrequest_id: 0, index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::ShardRateLimited as i32, + reason: PersistFailureReason::NoShardsAvailable as i32, }], + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), }); (persist_summary, persist_result) }); @@ -1277,108 +1167,20 @@ mod tests { )); } - #[tokio::test] - async fn test_router_process_persist_results_closes_and_deletes_shards() { - let self_node_id = "test-router".into(); - let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); - let ingester_pool = IngesterPool::default(); - let replication_factor = 1; - let router = IngestRouter::new( - self_node_id, - control_plane, - ingester_pool.clone(), - replication_factor, - EventBroker::default(), - ); - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - let mut state_guard = router.state.lock().await; - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![ - Shard { - index_uid: Some(index_uid.clone()), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), - ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - shard_id: Some(ShardId::from(2)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), - ..Default::default() - }, - ], - ); - drop(state_guard); - - let mut workbench = IngestWorkbench::new(Vec::new(), 2); - let persist_futures = FuturesUnordered::new(); - - persist_futures.push(async { - let persist_summary = PersistRequestSummary { - leader_id: "test-ingester-0".into(), - subrequest_ids: vec![0], - }; - let persist_result = Ok::<_, IngestV2Error>(PersistResponse { - leader_id: "test-ingester-0".to_string(), - successes: Vec::new(), - failures: vec![ - PersistFailure { - subrequest_id: 0, - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::ShardNotFound as i32, - }, - PersistFailure { - subrequest_id: 1, - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(2)), - reason: PersistFailureReason::ShardClosed as i32, - }, - ], - }); - (persist_summary, persist_result) - }); - router - .process_persist_results(&mut workbench, persist_futures) - .await; - - let state_guard = router.state.lock().await; - let routing_table_entry = state_guard - .routing_table - .find_entry("test-index-0", "test-source") - .unwrap(); - assert_eq!(routing_table_entry.len(), 1); - - let shard = routing_table_entry.all_shards()[0]; - assert_eq!(shard.shard_id, ShardId::from(2)); - assert_eq!(shard.shard_state, ShardState::Closed); - } - #[tokio::test] async fn test_router_process_persist_results_does_not_remove_unavailable_leaders() { let self_node_id = "test-router".into(); let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); - 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); + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry::mocked_ingester(), + ); + ingester_pool.insert( + "test-ingester-1".into(), + IngesterPoolEntry::mocked_ingester(), + ); let replication_factor = 1; let router = IngestRouter::new( @@ -1387,6 +1189,7 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); let ingest_subrequests = vec![ IngestSubrequest { @@ -1464,155 +1267,85 @@ mod tests { let self_node_id = "test-router".into(); let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); - let replication_factor = 1; let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), - replication_factor, + 1, EventBroker::default(), + Some("test-az".to_string()), ); - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - let index_uid2: IndexUid = IndexUid::for_test("test-index-1", 0); - let mut state_guard = router.state.lock().await; - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), - ..Default::default() - }], - ); - state_guard.routing_table.replace_shards( - index_uid2.clone(), - "test-source", - vec![ - Shard { - index_uid: Some(index_uid2.clone()), + + 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 mut state_guard = router.state.lock().await; + state_guard.routing_table.merge_from_shards( + index_uid_0.clone(), + "test-source".to_string(), + vec![Shard { + index_uid: Some(index_uid_0.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), - follower_id: Some("test-ingester-1".to_string()), ..Default::default() - }, - Shard { - index_uid: Some(index_uid2.clone()), + }], + ); + state_guard.routing_table.merge_from_shards( + index_uid_1.clone(), + "test-source".to_string(), + vec![Shard { + index_uid: Some(index_uid_1.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(2)), + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-1".to_string(), - follower_id: Some("test-ingester-2".to_string()), ..Default::default() - }, - ], - ); - drop(state_guard); + }], + ); + } + let index_uid_0_clone = index_uid_0.clone(); let mut mock_ingester_0 = MockIngesterService::new(); - let index_uid_clone = index_uid.clone(); - let index_uid2_clone = index_uid2.clone(); - mock_ingester_0 - .expect_persist() - .once() - .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.subrequests.len(), 2); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - assert_eq!(subrequest.index_uid(), &index_uid_clone); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["", "test-doc-foo", "test-doc-bar"])) - ); - - let subrequest = &request.subrequests[1]; - assert_eq!(subrequest.subrequest_id, 1); - assert_eq!(subrequest.index_uid(), &index_uid2_clone); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-qux"])) - ); - - let response = PersistResponse { - leader_id: request.leader_id, - successes: vec![ - PersistSuccess { - subrequest_id: 0, - index_uid: Some(index_uid_clone.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - replication_position_inclusive: Some(Position::offset(1u64)), - num_persisted_docs: 2, - parse_failures: vec![ParseFailure { - doc_uid: Some(DocUid::for_test(0)), - reason: ParseFailureReason::InvalidJson as i32, - message: "invalid JSON".to_string(), - }], - }, - PersistSuccess { - subrequest_id: 1, - index_uid: Some(index_uid2_clone.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - replication_position_inclusive: Some(Position::offset(0u64)), - num_persisted_docs: 1, - parse_failures: Vec::new(), - }, - ], - failures: Vec::new(), - }; - Ok(response) - }); mock_ingester_0 .expect_persist() .once() .returning(move |request| { assert_eq!(request.leader_id, "test-ingester-0"); assert_eq!(request.subrequests.len(), 1); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - assert_eq!(subrequest.index_uid(), &index_uid); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-moo", "test-doc-baz"])) - ); - let response = PersistResponse { + Ok(PersistResponse { leader_id: request.leader_id, successes: vec![PersistSuccess { subrequest_id: 0, - index_uid: Some(index_uid.clone()), + index_uid: Some(index_uid_0_clone.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), - replication_position_inclusive: Some(Position::offset(3u64)), - num_persisted_docs: 4, - parse_failures: Vec::new(), + replication_position_inclusive: Some(Position::offset(1u64)), + num_persisted_docs: 2, + parse_failures: vec![ParseFailure { + doc_uid: Some(DocUid::for_test(0)), + reason: ParseFailureReason::InvalidJson as i32, + message: "invalid JSON".to_string(), + }], }], failures: Vec::new(), - }; - Ok(response) + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), + }) }); - 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()); + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: IngesterServiceClient::from_mock(mock_ingester_0), + status: IngesterStatus::Ready, + availability_zone: None, + }, + ); let mut mock_ingester_1 = MockIngesterService::new(); mock_ingester_1 @@ -1621,171 +1354,129 @@ mod tests { .returning(move |request| { assert_eq!(request.leader_id, "test-ingester-1"); assert_eq!(request.subrequests.len(), 1); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 1); - assert_eq!(subrequest.index_uid(), &index_uid2); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(2)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-tux"])) - ); - let response = PersistResponse { + Ok(PersistResponse { leader_id: request.leader_id, successes: vec![PersistSuccess { subrequest_id: 1, - index_uid: Some(index_uid2.clone()), + index_uid: Some(index_uid_1.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(2)), + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), num_persisted_docs: 1, parse_failures: Vec::new(), }], failures: Vec::new(), - }; - Ok(response) + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), + }) }); - 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); + ingester_pool.insert( + "test-ingester-1".into(), + IngesterPoolEntry { + client: IngesterServiceClient::from_mock(mock_ingester_1), + availability_zone: None, + status: IngesterStatus::Ready, + }, + ); + + let response = router + .ingest(IngestRequestV2 { + subrequests: vec![ + IngestSubrequest { + subrequest_id: 0, + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2::for_test(["", "test-doc-foo", "test-doc-bar"])), + }, + IngestSubrequest { + subrequest_id: 1, + index_id: "test-index-1".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2::for_test(["test-doc-qux"])), + }, + ], + commit_type: CommitTypeV2::Auto as i32, + }) + .await + .unwrap(); - let ingest_request = IngestRequestV2 { - subrequests: vec![ - IngestSubrequest { - subrequest_id: 0, - index_id: "test-index-0".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["", "test-doc-foo", "test-doc-bar"])), - }, - IngestSubrequest { - subrequest_id: 1, - index_id: "test-index-1".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-qux"])), - }, - ], - commit_type: CommitTypeV2::Auto as i32, - }; - let response = router.ingest(ingest_request).await.unwrap(); assert_eq!(response.successes.len(), 2); assert_eq!(response.failures.len(), 0); let parse_failures = &response.successes[0].parse_failures; assert_eq!(parse_failures.len(), 1); - - let parse_failure = &parse_failures[0]; - assert_eq!(parse_failure.doc_uid(), DocUid::for_test(0)); - assert_eq!(parse_failure.reason(), ParseFailureReason::InvalidJson); - - let ingest_request = IngestRequestV2 { - subrequests: vec![ - IngestSubrequest { - subrequest_id: 0, - index_id: "test-index-0".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-moo", "test-doc-baz"])), - }, - IngestSubrequest { - subrequest_id: 1, - index_id: "test-index-1".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-tux"])), - }, - ], - commit_type: CommitTypeV2::Auto as i32, - }; - let response = router.ingest(ingest_request).await.unwrap(); - assert_eq!(response.successes.len(), 2); - assert_eq!(response.failures.len(), 0); - } + assert_eq!(parse_failures[0].doc_uid(), DocUid::for_test(0)); + assert_eq!(parse_failures[0].reason(), ParseFailureReason::InvalidJson); + } #[tokio::test] async fn test_router_ingest_retry() { let self_node_id = "test-router".into(); let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); - let replication_factor = 1; let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), - replication_factor, + 1, EventBroker::default(), + Some("test-az".to_string()), ); - let mut state_guard = router.state.lock().await; let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), - ..Default::default() - }], - ); - drop(state_guard); + { + let mut state_guard = router.state.lock().await; + state_guard.routing_table.merge_from_shards( + index_uid.clone(), + "test-source".to_string(), + vec![Shard { + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), + ..Default::default() + }], + ); + } let mut mock_ingester_0 = MockIngesterService::new(); + // First attempt: returns NoShardsAvailable (transient, doesn't mark leader unavailable). + // The response still reports capacity_score=6 and 1 open shard so the node stays routable. let index_uid_clone = index_uid.clone(); mock_ingester_0 .expect_persist() .once() .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.subrequests.len(), 1); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - assert_eq!(subrequest.index_uid(), &index_uid_clone); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); - - let response = PersistResponse { + Ok(PersistResponse { leader_id: request.leader_id, successes: Vec::new(), failures: vec![PersistFailure { subrequest_id: 0, index_uid: Some(index_uid_clone.clone()), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::Timeout as i32, + reason: PersistFailureReason::NoShardsAvailable as i32, }], - }; - Ok(response) + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: vec![SourceShardUpdate { + index_uid: Some(index_uid_clone.clone()), + source_id: "test-source".to_string(), + open_shard_count: 1, + }], + ..Default::default() + }), + }) }); + // Second attempt: succeeds. mock_ingester_0 .expect_persist() .once() .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.subrequests.len(), 1); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - assert_eq!(subrequest.index_uid(), &index_uid); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); - - let response = PersistResponse { + Ok(PersistResponse { leader_id: request.leader_id, successes: vec![PersistSuccess { subrequest_id: 0, @@ -1797,119 +1488,36 @@ mod tests { parse_failures: Vec::new(), }], failures: Vec::new(), - }; - Ok(response) + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), + }) }); - 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 { - subrequests: vec![IngestSubrequest { - subrequest_id: 0, - index_id: "test-index-0".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - }], - commit_type: CommitTypeV2::Auto as i32, - }; - router.ingest(ingest_request).await.unwrap(); - } - - #[tokio::test] - async fn test_router_updates_routing_table_on_chitchat_events() { - let self_node_id = "test-router".into(); - let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); - let ingester_pool = IngesterPool::default(); - let replication_factor = 1; - let event_broker = EventBroker::default(); - let router = IngestRouter::new( - self_node_id, - control_plane, - ingester_pool.clone(), - replication_factor, - event_broker.clone(), - ); - router.subscribe(); - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - - let mut state_guard = router.state.lock().await; - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid.clone()), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester".to_string(), - ..Default::default() - }], - ); - drop(state_guard); - - let local_shards_update = LocalShardsUpdate { - leader_id: "test-ingester".into(), - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: IngesterServiceClient::from_mock(mock_ingester_0), + status: IngesterStatus::Ready, + availability_zone: None, }, - shard_infos: BTreeSet::from_iter([ - ShardInfo { - shard_id: ShardId::from(1), - shard_state: ShardState::Closed, - short_term_ingestion_rate: RateMibPerSec(0), - long_term_ingestion_rate: RateMibPerSec(0), - }, - ShardInfo { - shard_id: ShardId::from(2), - shard_state: ShardState::Open, - short_term_ingestion_rate: RateMibPerSec(0), - long_term_ingestion_rate: RateMibPerSec(0), - }, - ]), - }; - event_broker.publish(local_shards_update); - - // Yield so that the event is processed. - yield_now().await; - - let state_guard = router.state.lock().await; - let shards = state_guard - .routing_table - .find_entry("test-index-0", "test-source") - .unwrap() - .all_shards(); - assert_eq!(shards.len(), 2); - assert_eq!(shards[0].shard_id, ShardId::from(1)); - assert_eq!(shards[0].shard_state, ShardState::Closed); - assert_eq!(shards[1].shard_id, ShardId::from(2)); - assert_eq!(shards[1].shard_state, ShardState::Open); - drop(state_guard); - - let shard_positions_update = ShardPositionsUpdate { - source_uid: SourceUid { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - }, - updated_shard_positions: vec![(ShardId::from(1), Position::eof(0u64))], - }; - event_broker.publish(shard_positions_update); - - // Yield so that the event is processed. - yield_now().await; + ); - let state_guard = router.state.lock().await; - let shards = state_guard - .routing_table - .find_entry("test-index-0", "test-source") - .unwrap() - .all_shards(); - assert_eq!(shards.len(), 1); - assert_eq!(shards[0].shard_id, ShardId::from(2)); - drop(state_guard); + let response = router + .ingest(IngestRequestV2 { + subrequests: vec![IngestSubrequest { + subrequest_id: 0, + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), + }], + commit_type: CommitTypeV2::Auto as i32, + }) + .await + .unwrap(); + assert_eq!(response.successes.len(), 1); + assert_eq!(response.failures.len(), 0); } #[tokio::test] @@ -1924,46 +1532,53 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); 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 mut state_guard = router.state.lock().await; - state_guard.routing_table.replace_shards( - index_uid_0.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid_0.clone()), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester".to_string(), - ..Default::default() - }], - ); - state_guard.routing_table.replace_shards( - index_uid_1.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid_1.clone()), - shard_id: Some(ShardId::from(2)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester".to_string(), - ..Default::default() - }], - ); - drop(state_guard); + { + let mut state_guard = router.state.lock().await; + state_guard.routing_table.merge_from_shards( + index_uid_0.clone(), + "test-source".to_string(), + vec![Shard { + index_uid: Some(index_uid_0.clone()), + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), + ..Default::default() + }], + ); + state_guard.routing_table.merge_from_shards( + index_uid_1.clone(), + "test-source".to_string(), + vec![Shard { + index_uid: Some(index_uid_1.clone()), + shard_id: Some(ShardId::from(2)), + shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), + ..Default::default() + }], + ); + } let debug_info = router.debug_info().await; let routing_table = &debug_info["routing_table"]; assert_eq!(routing_table.as_object().unwrap().len(), 2); - assert_eq!(routing_table["test-index-0"].as_array().unwrap().len(), 1); - assert_eq!(routing_table["test-index-1"].as_array().unwrap().len(), 1); + let index_0_entries = routing_table["test-index-0"].as_array().unwrap(); + assert_eq!(index_0_entries.len(), 1); + assert_eq!(index_0_entries[0]["node_id"], "test-ingester-0"); + assert_eq!(index_0_entries[0]["capacity_score"], 5); + + let index_1_entries = routing_table["test-index-1"].as_array().unwrap(); + assert_eq!(index_1_entries.len(), 1); + assert_eq!(index_1_entries[0]["node_id"], "test-ingester-1"); } #[tokio::test] - async fn test_router_does_not_retry_rate_limited_shards() { - // We avoid retrying a shard limited shard at the scale of a workbench. + async fn test_router_returns_rate_limited_failure() { let self_node_id = "test-router".into(); let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); @@ -1974,145 +1589,69 @@ mod tests { ingester_pool.clone(), replication_factor, EventBroker::default(), + Some("test-az".to_string()), ); - let mut state_guard = router.state.lock().await; let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); - - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![ - Shard { + { + let mut state_guard = router.state.lock().await; + state_guard.routing_table.merge_from_shards( + index_uid.clone(), + "test-source".to_string(), + vec![Shard { index_uid: Some(index_uid.clone()), source_id: "test-source".to_string(), shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(2)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), - ..Default::default() - }, - ], - ); - drop(state_guard); - - // We have two shards. - // - shard 1 is rate limited - // - shard 2 is timeout. - // We expect a retry on shard 2 that is then successful. - let mut seq = Sequence::new(); + }], + ); + } let mut mock_ingester_0 = MockIngesterService::new(); - mock_ingester_0 - .expect_persist() - .times(1) - .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - assert_eq!(request.subrequests.len(), 1); - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - let index_uid = subrequest.index_uid().clone(); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); - - let response = PersistResponse { - leader_id: request.leader_id, - successes: Vec::new(), - failures: vec![PersistFailure { - subrequest_id: 0, - index_uid: Some(index_uid), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::ShardRateLimited as i32, - }], - }; - Ok(response) - }) - .in_sequence(&mut seq); - - mock_ingester_0 - .expect_persist() - .times(1) - .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - assert_eq!(request.subrequests.len(), 1); - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - let index_uid = subrequest.index_uid().clone(); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(2)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); - - let response = PersistResponse { - leader_id: request.leader_id, - successes: Vec::new(), - failures: vec![PersistFailure { - subrequest_id: 0, - index_uid: Some(index_uid), - source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::Timeout as i32, - }], - }; - Ok(response) - }) - .in_sequence(&mut seq); - - mock_ingester_0 - .expect_persist() - .times(1) - .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - assert_eq!(request.subrequests.len(), 1); - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - let index_uid = subrequest.index_uid().clone(); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(2)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); + mock_ingester_0.expect_persist().returning(move |request| { + assert_eq!(request.leader_id, "test-ingester-0"); + assert_eq!(request.commit_type(), CommitTypeV2::Auto); + assert_eq!(request.subrequests.len(), 1); + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.subrequest_id, 0); + let index_uid = subrequest.index_uid().clone(); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!( + subrequest.doc_batch, + Some(DocBatchV2::for_test(["test-doc-foo"])) + ); - let response = PersistResponse { - leader_id: request.leader_id, - successes: vec![PersistSuccess { - subrequest_id: 0, + let response = PersistResponse { + leader_id: request.leader_id, + successes: Vec::new(), + failures: vec![PersistFailure { + subrequest_id: 0, + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + reason: PersistFailureReason::NoShardsAvailable as i32, + }], + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: vec![SourceShardUpdate { index_uid: Some(index_uid), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - num_persisted_docs: 1, - replication_position_inclusive: Some(Position::offset(0u64)), - parse_failures: Vec::new(), + open_shard_count: 1, }], - failures: Vec::new(), - }; - Ok(response) - }) - .in_sequence(&mut seq); - - 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()); + ..Default::default() + }), + }; + Ok(response) + }); + let ingester_0 = IngesterServiceClient::from_mock(mock_ingester_0); + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry { + client: ingester_0.clone(), + availability_zone: None, + status: IngesterStatus::Ready, + }, + ); let ingest_request = IngestRequestV2 { subrequests: vec![IngestSubrequest { @@ -2123,92 +1662,200 @@ mod tests { }], commit_type: CommitTypeV2::Auto as i32, }; - router.ingest(ingest_request).await.unwrap(); + let ingest_response = router.ingest(ingest_request).await.unwrap(); + assert_eq!(ingest_response.successes.len(), 0); + assert_eq!(ingest_response.failures.len(), 1); + assert_eq!( + ingest_response.failures[0].reason(), + IngestFailureReason::NoShardsAvailable + ); } #[tokio::test] - async fn test_router_returns_rate_limited_failure() { - let self_node_id = "test-router".into(); - let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); + async fn test_router_updates_node_routing_table_on_capacity_update() { + let event_broker = EventBroker::default(); let ingester_pool = IngesterPool::default(); - let replication_factor = 1; let router = IngestRouter::new( - self_node_id, - control_plane, + "test-router".into(), + ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()), ingester_pool.clone(), - replication_factor, - EventBroker::default(), + 1, + event_broker.clone(), + Some("test-az".to_string()), ); - let mut state_guard = router.state.lock().await; - let index_uid: IndexUid = IndexUid::for_test("test-index-0", 0); + router.subscribe(); - state_guard.routing_table.replace_shards( - index_uid.clone(), - "test-source", - vec![Shard { - index_uid: Some(index_uid.clone()), + event_broker.publish(IngesterCapacityScoreUpdate { + node_id: "test-ingester-0".into(), + source_uid: SourceUid { + index_uid: IndexUid::for_test("test-index", 0), + source_id: "test-source".to_string(), + }, + capacity_score: 7, + open_shard_count: 3, + }); + // Give the async subscriber a moment to process. + tokio::time::sleep(Duration::from_millis(10)).await; + + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry::mocked_ingester(), + ); + let state_guard = router.state.lock().await; + let node = state_guard + .routing_table + .pick_node("test-index", "test-source", &ingester_pool, &HashSet::new()) + .unwrap(); + assert_eq!(node.node_id, NodeId::from("test-ingester-0")); + } + + #[tokio::test] + async fn test_router_process_persist_results_marks_unavailable_on_persist_failure() { + let router = IngestRouter::new( + "test-router".into(), + ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()), + IngesterPool::default(), + 1, + EventBroker::default(), + Some("test-az".to_string()), + ); + let ingest_subrequests = vec![ + IngestSubrequest { + subrequest_id: 0, + index_id: "test-index-0".to_string(), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-ingester-0".to_string(), ..Default::default() - }], + }, + IngestSubrequest { + subrequest_id: 1, + index_id: "test-index-1".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + ]; + let mut workbench = IngestWorkbench::new(ingest_subrequests, 2); + + // NoShardsAvailable does NOT mark the leader as unavailable. + let persist_futures = FuturesUnordered::new(); + persist_futures.push(async { + let summary = PersistRequestSummary { + leader_id: "test-ingester-0".into(), + subrequest_ids: vec![0], + }; + let result = Ok::<_, IngestV2Error>(PersistResponse { + leader_id: "test-ingester-0".to_string(), + successes: Vec::new(), + failures: vec![PersistFailure { + subrequest_id: 0, + index_uid: Some(IndexUid::for_test("test-index-0", 0)), + source_id: "test-source".to_string(), + reason: PersistFailureReason::NoShardsAvailable as i32, + }], + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), + }); + (summary, result) + }); + router + .process_persist_results(&mut workbench, persist_futures) + .await; + assert!( + !workbench + .unavailable_leaders + .contains(&NodeId::from("test-ingester-0")) ); - drop(state_guard); - let mut mock_ingester_0 = MockIngesterService::new(); - mock_ingester_0 - .expect_persist() - .times(1) - .returning(move |request| { - assert_eq!(request.leader_id, "test-ingester-0"); - assert_eq!(request.commit_type(), CommitTypeV2::Auto); - assert_eq!(request.subrequests.len(), 1); - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.subrequest_id, 0); - let index_uid = subrequest.index_uid().clone(); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id(), ShardId::from(1)); - assert_eq!( - subrequest.doc_batch, - Some(DocBatchV2::for_test(["test-doc-foo"])) - ); + // NodeUnavailable DOES mark the leader as unavailable. + let persist_futures = FuturesUnordered::new(); + persist_futures.push(async { + let summary = PersistRequestSummary { + leader_id: "test-ingester-1".into(), + subrequest_ids: vec![1], + }; + let result = Ok::<_, IngestV2Error>(PersistResponse { + leader_id: "test-ingester-1".to_string(), + successes: Vec::new(), + failures: vec![PersistFailure { + subrequest_id: 1, + index_uid: Some(IndexUid::for_test("test-index-1", 0)), + source_id: "test-source".to_string(), + reason: PersistFailureReason::NodeUnavailable as i32, + }], + routing_update: Some(RoutingUpdate { + capacity_score: 6, + source_shard_updates: Vec::new(), + ..Default::default() + }), + }); + (summary, result) + }); + router + .process_persist_results(&mut workbench, persist_futures) + .await; + assert!( + workbench + .unavailable_leaders + .contains(&NodeId::from("test-ingester-1")) + ); + } - let response = PersistResponse { - leader_id: request.leader_id, - successes: Vec::new(), - failures: vec![PersistFailure { - subrequest_id: 0, - index_uid: Some(index_uid), + #[tokio::test] + async fn test_router_process_persist_results_applies_piggybacked_routing_updates() { + let ingester_pool = IngesterPool::default(); + let router = IngestRouter::new( + "test-router".into(), + ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()), + ingester_pool.clone(), + 1, + EventBroker::default(), + Some("test-az".to_string()), + ); + let ingest_subrequests = vec![IngestSubrequest { + subrequest_id: 0, + index_id: "test-index".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }]; + let mut workbench = IngestWorkbench::new(ingest_subrequests, 2); + + let persist_futures = FuturesUnordered::new(); + persist_futures.push(async { + let summary = PersistRequestSummary { + leader_id: "test-ingester-0".into(), + subrequest_ids: vec![0], + }; + let result = Ok::<_, IngestV2Error>(PersistResponse { + leader_id: "test-ingester-0".to_string(), + successes: Vec::new(), + failures: Vec::new(), + routing_update: Some(RoutingUpdate { + capacity_score: 3, + source_shard_updates: vec![SourceShardUpdate { + index_uid: Some(IndexUid::for_test("test-index", 0)), source_id: "test-source".to_string(), - shard_id: Some(ShardId::from(1)), - reason: PersistFailureReason::ShardRateLimited as i32, + open_shard_count: 2, }], - }; - Ok(response) + ..Default::default() + }), }); - 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()); + (summary, result) + }); + router + .process_persist_results(&mut workbench, persist_futures) + .await; - let ingest_request = IngestRequestV2 { - subrequests: vec![IngestSubrequest { - subrequest_id: 0, - index_id: "test-index-0".to_string(), - source_id: "test-source".to_string(), - doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - }], - commit_type: CommitTypeV2::Auto as i32, - }; - let ingest_response = router.ingest(ingest_request).await.unwrap(); - assert_eq!(ingest_response.successes.len(), 0); - assert_eq!(ingest_response.failures.len(), 1); - assert_eq!( - ingest_response.failures[0].reason(), - IngestFailureReason::ShardRateLimited + ingester_pool.insert( + "test-ingester-0".into(), + IngesterPoolEntry::mocked_ingester(), ); + let state_guard = router.state.lock().await; + let node = state_guard + .routing_table + .pick_node("test-index", "test-source", &ingester_pool, &HashSet::new()) + .unwrap(); + assert_eq!(node.node_id, NodeId::from("test-ingester-0")); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs index 3eef8788f64..f5475387287 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs @@ -12,492 +12,244 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::hash_map::Entry; use std::collections::{HashMap, HashSet}; -use std::sync::atomic::{AtomicUsize, Ordering}; -use quickwit_proto::ingest::{Shard, ShardIds, ShardState}; -use quickwit_proto::types::{IndexId, IndexUid, NodeId, ShardId, SourceId}; -use serde_json::{Value as JsonValue, json}; -use tracing::{info, warn}; +use itertools::Itertools; +use quickwit_proto::ingest::Shard; +use quickwit_proto::types::{IndexId, IndexUid, NodeId, SourceId}; +use rand::rng; +use rand::seq::IndexedRandom; use crate::IngesterPool; -#[derive(Debug)] -pub(super) struct RoutingEntry { +/// A single ingester node's routing-relevant data for a specific (index, source) pair. +/// Each entry is self-describing: it carries its own node_id, index_uid, and source_id +/// so it can always be attributed back to a specific source on a specific node. +#[derive(Debug, Clone)] +pub(super) struct IngesterNode { + pub node_id: NodeId, pub index_uid: IndexUid, + #[allow(unused)] pub source_id: SourceId, - pub shard_id: ShardId, - pub shard_state: ShardState, - pub leader_id: NodeId, + /// Score from 0-10. Higher means more available capacity. + pub capacity_score: usize, + /// Number of open shards on this node for this (index, source) pair. Tiebreaker for power of + /// two choices comparison - we favor a node with more open shards. + pub open_shard_count: usize, } -impl From for RoutingEntry { - fn from(shard: Shard) -> Self { - let shard_id = shard.shard_id().clone(); - let shard_state = shard.shard_state(); - Self { - index_uid: shard.index_uid().clone(), - source_id: shard.source_id, - shard_id, - shard_state, - leader_id: shard.leader_id.into(), - } - } -} - -/// The set of shards the router is aware of for the given index and source. #[derive(Debug, Default)] -pub(super) struct RoutingTableEntry { - /// Index UID of the shards. - pub index_uid: IndexUid, - /// Source ID of the shards. - pub source_id: SourceId, - /// Shards located on this node. - pub local_shards: Vec, - pub local_round_robin_idx: AtomicUsize, - /// Shards located on remote nodes. - pub remote_shards: Vec, - pub remote_round_robin_idx: AtomicUsize, +pub(super) struct RoutingEntry { + pub nodes: HashMap, } -impl RoutingTableEntry { - /// Creates a new entry and ensures that the shards are open, unique, and sorted by shard ID. - fn new( - self_node_id: &NodeId, - index_uid: IndexUid, - source_id: SourceId, - mut shards: Vec, - ) -> Self { - let num_shards = shards.len(); - - shards.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); - shards.dedup_by(|left, right| left.shard_id == right.shard_id); +/// Given a slice of candidates, picks the better of two random choices. +/// Higher capacity_score wins; tiebreak on more open_shard_count (more landing spots). +fn power_of_two_choices<'a>(candidates: &[&'a IngesterNode]) -> &'a IngesterNode { + debug_assert!(candidates.len() >= 2); + let mut iter = candidates.choose_multiple(&mut rng(), 2); + let (&a, &b) = (iter.next().unwrap(), iter.next().unwrap()); + + if (a.capacity_score, a.open_shard_count) >= (b.capacity_score, b.open_shard_count) { + a + } else { + b + } +} - let (local_shards, remote_shards): (Vec<_>, Vec<_>) = shards - .into_iter() - .filter(|shard| shard.is_open()) - .map(RoutingEntry::from) - .partition(|shard| *self_node_id == shard.leader_id); +fn pick_from(candidates: Vec<&IngesterNode>) -> Option<&IngesterNode> { + match candidates.len() { + 0 => None, + 1 => Some(candidates[0]), + _ => Some(power_of_two_choices(&candidates)), + } +} - if num_shards > local_shards.len() + remote_shards.len() { - warn!("input shards should not contain closed shards or duplicates"); - } +impl RoutingEntry { + /// Pick an ingester node to persist the request to. Uses power of two choices based on reported + /// ingester capacity, if more than one eligible node exists. Prefers nodes in the same + /// availability zone, falling back to remote nodes. + fn pick_node( + &self, + ingester_pool: &IngesterPool, + unavailable_leaders: &HashSet, + self_availability_zone: &Option, + ) -> Option<&IngesterNode> { + let (local_ingesters, remote_ingesters): (Vec<&IngesterNode>, Vec<&IngesterNode>) = self + .nodes + .values() + .filter(|node| { + node.capacity_score > 0 + && node.open_shard_count > 0 + && ingester_pool + .get(&node.node_id) + .map(|entry| entry.status.is_ready()) + .unwrap_or(false) + && !unavailable_leaders.contains(&node.node_id) + }) + .partition(|node| { + let node_az = ingester_pool + .get(&node.node_id) + .and_then(|h| h.availability_zone); + node_az == *self_availability_zone + }); - Self { - index_uid, - source_id, - local_shards, - remote_shards, - ..Default::default() - } + pick_from(local_ingesters).or_else(|| pick_from(remote_ingesters)) } +} - fn empty(index_uid: IndexUid, source_id: SourceId) -> Self { +#[derive(Debug, Default)] +pub(super) struct RoutingTable { + table: HashMap<(IndexId, SourceId), RoutingEntry>, + self_availability_zone: Option, +} + +impl RoutingTable { + pub fn new(self_availability_zone: Option) -> Self { Self { - index_uid, - source_id, + self_availability_zone, ..Default::default() } } - /// Returns `true` if at least one shard in the table entry is open and has a leader available. - /// As it goes through the list of shards in the entry, it populates `closed_shard_ids` and - /// `unavailable_leaders` with the shard IDs of the closed shards and the node ID of the - /// unavailable ingesters encountered along the way. - pub fn has_open_shards( + pub fn pick_node( &self, + index_id: &str, + source_id: &str, ingester_pool: &IngesterPool, - closed_shard_ids: &mut Vec, - unavailable_leaders: &mut HashSet, - ) -> bool { - let shards = self.local_shards.iter().chain(self.remote_shards.iter()); - - for shard in shards { - match shard.shard_state { - ShardState::Closed => { - closed_shard_ids.push(shard.shard_id.clone()); - continue; - } - ShardState::Unavailable | ShardState::Unspecified => { - continue; - } - ShardState::Open => { - if unavailable_leaders.contains(&shard.leader_id) { - continue; - } - 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; - } - } - } - false + unavailable_leaders: &HashSet, + ) -> Option<&IngesterNode> { + let key = (index_id.to_string(), source_id.to_string()); + let entry = self.table.get(&key)?; + entry.pick_node( + ingester_pool, + unavailable_leaders, + &self.self_availability_zone, + ) } - /// Returns the next open and available shard in the table entry in a round-robin fashion. - pub fn next_open_shard_round_robin( + pub fn classify_az_locality( &self, + target_node_id: &NodeId, ingester_pool: &IngesterPool, - rate_limited_shards: &HashSet, - ) -> Result<&RoutingEntry, NextOpenShardError> { - let mut error = NextOpenShardError::NoShardsAvailable; - - for (shards, round_robin_idx) in [ - (&self.local_shards, &self.local_round_robin_idx), - (&self.remote_shards, &self.remote_round_robin_idx), - ] { - if shards.is_empty() { - continue; - } - for _attempt in 0..shards.len() { - let shard_idx = round_robin_idx.fetch_add(1, Ordering::Relaxed); - let shard_routing_entry: &RoutingEntry = &shards[shard_idx % shards.len()]; - - if !shard_routing_entry.shard_state.is_open() { - continue; - } - if rate_limited_shards.contains(&shard_routing_entry.shard_id) { - error = NextOpenShardError::RateLimited; - continue; - } - if let Some(ingester) = ingester_pool.get(&shard_routing_entry.leader_id) - && ingester.status.is_ready() - { - return Ok(shard_routing_entry); - } - } - } - Err(error) - } - - /// Inserts the open shards the routing table is not aware of. - fn insert_open_shards( - &mut self, - self_node_id: &NodeId, - leader_id: &NodeId, - index_uid: &IndexUid, - shard_ids: &[ShardId], - ) { - match self.index_uid.cmp(index_uid) { - // If we receive an update for a new incarnation of the index, then we clear the entry - // and insert all the shards. - std::cmp::Ordering::Less => { - self.index_uid = index_uid.clone(); - self.clear_shards(); - } - // If we receive an update for a previous incarnation of the index, then we ignore it. - std::cmp::Ordering::Greater => { - return; - } - std::cmp::Ordering::Equal => {} + ) -> &'static str { + let Some(self_az) = &self.self_availability_zone else { + return "az_unaware"; }; - let target_shards = if self_node_id == leader_id { - &mut self.local_shards - } else { - &mut self.remote_shards - }; - let mut num_inserted_shards = 0; - let num_target_shards = target_shards.len(); - - if num_target_shards == 0 { - target_shards.reserve(num_target_shards); - target_shards.extend(shard_ids.iter().map(|shard_id| RoutingEntry { - index_uid: self.index_uid.clone(), - source_id: self.source_id.clone(), - shard_id: shard_id.clone(), - shard_state: ShardState::Open, - leader_id: leader_id.clone(), - })); - num_inserted_shards = target_shards.len(); - } else { - let shard_ids_range = target_shards[0].shard_id.clone() - ..=target_shards[num_target_shards - 1].shard_id.clone(); - - for shard_id in shard_ids { - // If we can't find the shard, then we insert it. - if shard_ids_range.contains(shard_id) { - continue; - } - if target_shards[..num_target_shards] - .binary_search_by(|shard| shard.shard_id.cmp(shard_id)) - .is_err() - { - target_shards.push(RoutingEntry { - index_uid: self.index_uid.clone(), - source_id: self.source_id.clone(), - shard_id: shard_id.clone(), - shard_state: ShardState::Open, - leader_id: leader_id.clone(), - }); - num_inserted_shards += 1; - } - } - } - if num_inserted_shards > 0 { - target_shards.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); - - info!( - index_uid=%self.index_uid, - source_id=%self.source_id, - "inserted {num_inserted_shards} shards into routing table" - ); + let target_az = ingester_pool + .get(target_node_id) + .and_then(|entry| entry.availability_zone); + match target_az { + Some(ref az) if az == self_az => "same_az", + Some(_) => "cross_az", + None => "az_unaware", } } - /// Clears local and remote shards. - fn clear_shards(&mut self) { - self.local_shards.clear(); - self.local_round_robin_idx = AtomicUsize::default(); - self.remote_shards.clear(); - self.remote_round_robin_idx = AtomicUsize::default(); - } - - /// Closes the shards identified by their shard IDs. - fn close_shards(&mut self, index_uid: &IndexUid, shard_ids: &[ShardId]) { - // If the shard table was just recently updated with shards for a new index UID, then we can - // safely discard this request. - if self.index_uid != *index_uid { - return; - } - for shards in [&mut self.local_shards, &mut self.remote_shards] { - if shards.is_empty() { - continue; - } - let num_shards = shards.len(); - let shard_ids_range = - shards[0].shard_id.clone()..=shards[num_shards - 1].shard_id.clone(); - - for shard_id in shard_ids { - if !shard_ids_range.contains(shard_id) { - continue; - } - if let Ok(shard_idx) = shards.binary_search_by(|shard| shard.shard_id.cmp(shard_id)) - { - shards[shard_idx].shard_state = ShardState::Closed; - } - } - } - } - - /// Shards the shards identified by their shard IDs. - fn delete_shards(&mut self, index_uid: &IndexUid, shard_ids: &[ShardId]) { - // If the shard table was just recently updated with shards for a new index UID, then we can - // safely discard this request. - if self.index_uid != *index_uid { - return; - } - for shards in [&mut self.local_shards, &mut self.remote_shards] { - if shards.is_empty() { - continue; - } - let num_shards = shards.len(); - let shard_ids_range = - shards[0].shard_id.clone()..=shards[num_shards - 1].shard_id.clone(); - let mut deleted_any = false; - - for shard_id in shard_ids { - if !shard_ids_range.contains(shard_id) { - continue; - } - if let Ok(shard_idx) = shards.binary_search_by(|shard| shard.shard_id.cmp(shard_id)) - { - // We use `Unspecified` as a tombstone. - shards[shard_idx].shard_state = ShardState::Unspecified; - deleted_any = true; - } - } - if deleted_any { - shards.retain(|shard| shard.shard_state != ShardState::Unspecified); + pub fn debug_info( + &self, + ingester_pool: &IngesterPool, + ) -> HashMap> { + let mut per_index: HashMap> = HashMap::new(); + for ((index_id, source_id), entry) in &self.table { + for (node_id, node) in &entry.nodes { + let az = ingester_pool.get(node_id).and_then(|h| h.availability_zone); + per_index + .entry(index_id.clone()) + .or_default() + .push(serde_json::json!({ + "source_id": source_id, + "node_id": node_id, + "capacity_score": node.capacity_score, + "open_shard_count": node.open_shard_count, + "availability_zone": az, + })); } } + per_index } - #[cfg(test)] - pub fn len(&self) -> usize { - self.local_shards.len() + self.remote_shards.len() - } - - #[cfg(test)] - pub fn all_shards(&self) -> Vec<&RoutingEntry> { - let mut shards = Vec::with_capacity(self.len()); - shards.extend(&self.local_shards); - shards.extend(&self.remote_shards); - shards - } -} - -#[derive(Debug, PartialEq, Eq)] -pub(super) enum NextOpenShardError { - NoShardsAvailable, - RateLimited, -} - -/// Stores the list of shards the router is aware of for each index and source. The resolution from -/// index and source to shards is performed using index ID (not index UID) and source ID. -#[derive(Debug)] -pub(super) struct RoutingTable { - pub self_node_id: NodeId, - pub table: HashMap<(IndexId, SourceId), RoutingTableEntry>, -} - -impl RoutingTable { - pub fn find_entry( - &self, - index_id: impl Into, - source_id: impl Into, - ) -> Option<&RoutingTableEntry> { - let key = (index_id.into(), source_id.into()); - self.table.get(&key) - } - - /// Returns `true` if the router already knows about a shard for a given source that has - /// an available `leader`. - /// - /// If this function returns false, it populates the set of unavailable leaders and closed - /// shards. These will be joined to the GetOrCreate shard request emitted to the control - /// plane. - pub fn has_open_shards( + pub fn has_open_nodes( &self, - index_id: impl Into, - source_id: impl Into, + index_id: &str, + source_id: &str, ingester_pool: &IngesterPool, - closed_shards: &mut Vec, - unavailable_leaders: &mut HashSet, + unavailable_leaders: &HashSet, ) -> bool { - let Some(entry) = self.find_entry(index_id, source_id) else { + let key = (index_id.to_string(), source_id.to_string()); + let Some(entry) = self.table.get(&key) else { return false; }; - let mut closed_shard_ids: Vec = Vec::new(); - - let result = - entry.has_open_shards(ingester_pool, &mut closed_shard_ids, unavailable_leaders); - - if !closed_shard_ids.is_empty() { - closed_shards.push(ShardIds { - index_uid: entry.index_uid.clone().into(), - source_id: entry.source_id.clone(), - shard_ids: closed_shard_ids, - }); - } - result + entry.nodes.values().any(|node| { + node.capacity_score > 0 + && node.open_shard_count > 0 + && ingester_pool + .get(&node.node_id) + .map(|entry| entry.status.is_ready()) + .unwrap_or(false) + && !unavailable_leaders.contains(&node.node_id) + }) } - /// Replaces the routing table entry for the source with the provided shards. - pub fn replace_shards( + /// Applies a capacity update from the IngesterCapacityScoreUpdate broadcast. This is the + /// primary way the table learns about node availability and capacity. + pub fn apply_capacity_update( &mut self, + node_id: NodeId, index_uid: IndexUid, - source_id: impl Into, - shards: Vec, + source_id: SourceId, + capacity_score: usize, + open_shard_count: usize, ) { - let index_id: IndexId = index_uid.index_id.to_string(); - let source_id: SourceId = source_id.into(); - let key = (index_id, source_id.clone()); + let key = (index_uid.index_id.to_string(), source_id.clone()); - match self.table.entry(key) { - Entry::Vacant(entry) => { - entry.insert(RoutingTableEntry::new( - &self.self_node_id, - index_uid, - source_id, - shards, - )); - } - Entry::Occupied(mut entry) => { - assert!( - entry.get().index_uid <= index_uid, - "new index incarnation should be greater or equal" - ); - - entry.insert(RoutingTableEntry::new( - &self.self_node_id, - index_uid, - source_id, - shards, - )); - } + let entry = self.table.entry(key).or_default(); + let ingester_node = IngesterNode { + node_id: node_id.clone(), + index_uid, + source_id, + capacity_score, + open_shard_count, }; + entry.nodes.insert(node_id, ingester_node); } - /// Inserts the shards the routing table is not aware of. - pub fn insert_open_shards( + /// Merges routing updates from a GetOrCreateOpenShards control plane response into the + /// table. For existing nodes, updates their open shard count, including if the count is 0, from + /// the CP response while preserving capacity scores if they already exist. + /// New nodes get a default capacity_score of 5. + pub fn merge_from_shards( &mut self, - leader_id: &NodeId, index_uid: IndexUid, - source_id: impl Into, - shard_ids: &[ShardId], - ) { - let index_id: IndexId = index_uid.index_id.to_string(); - let source_id: SourceId = source_id.into(); - let key = (index_id, source_id.clone()); - - self.table - .entry(key.clone()) - .or_insert_with(|| RoutingTableEntry::empty(index_uid.clone(), source_id)) - .insert_open_shards(&self.self_node_id, leader_id, &index_uid, shard_ids); - } - - /// Closes the targeted shards. - pub fn close_shards( - &mut self, - index_uid: &IndexUid, - source_id: impl Into, - shard_ids: &[ShardId], - ) { - let key = (index_uid.index_id.clone(), source_id.into()); - if let Some(entry) = self.table.get_mut(&key) { - entry.close_shards(index_uid, shard_ids); - } - } - - /// Deletes the targeted shards. - pub fn delete_shards( - &mut self, - index_uid: &IndexUid, - source_id: impl Into, - shard_ids: &[ShardId], + source_id: SourceId, + shards: Vec, ) { - let key = (index_uid.index_id.clone(), source_id.into()); - if let Some(entry) = self.table.get_mut(&key) { - entry.delete_shards(index_uid, shard_ids); - } - } - - pub fn debug_info(&self) -> HashMap> { - let mut per_index_shards_json: HashMap> = HashMap::new(); - - for ((index_id, source_id), entry) in &self.table { - for (shards, is_local) in &[(&entry.local_shards, true), (&entry.remote_shards, false)] - { - let shards_json = shards.iter().map(|shard| { - json!({ - "index_uid": shard.index_uid, - "source_id": source_id, - "shard_id": shard.shard_id, - "shard_state": shard.shard_state.as_json_str_name(), - "is_local": is_local, - }) + let per_leader_count: HashMap = shards + .iter() + .map(|shard| { + let num_open_shards = shard.is_open() as usize; + let leader_id = NodeId::from(shard.leader_id.clone()); + (leader_id, num_open_shards) + }) + .into_grouping_map() + .sum(); + + let key = (index_uid.index_id.to_string(), source_id.clone()); + let entry = self.table.entry(key).or_default(); + + for (node_id, open_shard_count) in per_leader_count { + entry + .nodes + .entry(node_id.clone()) + .and_modify(|node| node.open_shard_count = open_shard_count) + .or_insert_with(|| IngesterNode { + node_id, + index_uid: index_uid.clone(), + source_id: source_id.clone(), + capacity_score: 5, + open_shard_count, }); - per_index_shards_json - .entry(index_id.clone()) - .or_default() - .extend(shards_json); - } } - per_index_shards_json - } - - #[cfg(test)] - pub fn len(&self) -> usize { - self.table.len() } } @@ -505,639 +257,316 @@ impl RoutingTable { mod tests { use quickwit_proto::ingest::ShardState; use quickwit_proto::ingest::ingester::{IngesterServiceClient, IngesterStatus}; + use quickwit_proto::types::ShardId; 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::from("test-source"); - let table_entry = RoutingTableEntry::new( - &self_node_id, - index_uid.clone(), - source_id.clone(), - Vec::new(), - ); - assert_eq!(table_entry.len(), 0); - - let index_uid: IndexUid = IndexUid::for_test("test-index", 0); - let shards = vec![ - Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(3)), - shard_state: ShardState::Open as i32, - leader_id: "test-node-0".to_string(), - ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-node-0".to_string(), - ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(2)), - shard_state: ShardState::Open as i32, - leader_id: "test-node-1".to_string(), - ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(1)), - shard_state: ShardState::Open as i32, - leader_id: "test-node-0".to_string(), - ..Default::default() - }, - Shard { - index_uid: Some(index_uid.clone()), - source_id: source_id.clone(), - shard_id: Some(ShardId::from(4)), - shard_state: ShardState::Closed as i32, - leader_id: "test-node-0".to_string(), - ..Default::default() - }, - ]; - let table_entry = RoutingTableEntry::new(&self_node_id, index_uid, source_id, shards); - assert_eq!(table_entry.local_shards.len(), 2); - assert_eq!(table_entry.local_shards[0].shard_id, ShardId::from(1)); - assert_eq!(table_entry.local_shards[1].shard_id, ShardId::from(3)); - - assert_eq!(table_entry.remote_shards.len(), 1); - assert_eq!(table_entry.remote_shards[0].shard_id, ShardId::from(2)); + fn mocked_ingester(availability_zone: Option<&str>) -> IngesterPoolEntry { + IngesterPoolEntry { + client: IngesterServiceClient::mocked(), + status: IngesterStatus::Ready, + availability_zone: availability_zone.map(|s| s.to_string()), + } } #[test] - fn test_routing_table_entry_has_open_shards() { - let index_uid = IndexUid::for_test("test-index", 0); - 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(); - let ingester_pool = IngesterPool::default(); - let mut unavailable_leaders = HashSet::new(); - - assert!(!table_entry.has_open_shards( - &ingester_pool, - &mut closed_shard_ids, - &mut unavailable_leaders - )); - assert!(closed_shard_ids.is_empty()); - assert!(unavailable_leaders.is_empty()); - - // Ingester 0 is ready, but shard 0 is closed. - ingester_pool.insert( - "test-ingester-0".into(), - IngesterPoolEntry { - client: IngesterServiceClient::mocked(), - status: IngesterStatus::Ready, - }, + fn test_apply_capacity_update() { + let mut table = RoutingTable::default(); + let key = ("test-index".to_string(), "test-source".into()); + + // Insert first node. + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 8, + 3, ); - // 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, - }, + let entry = table.table.get(&key).unwrap(); + assert_eq!(entry.nodes.len(), 1); + assert_eq!(entry.nodes.get("node-1").unwrap().capacity_score, 8); + + // Update existing node. + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 4, + 5, ); - // 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, - }, + let node = table.table.get(&key).unwrap().nodes.get("node-1").unwrap(); + assert_eq!(node.capacity_score, 4); + assert_eq!(node.open_shard_count, 5); + + // Add second node. + table.apply_capacity_update( + "node-2".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 6, + 2, ); - // Ingester 4 is ready, and shard 4 is open. - ingester_pool.insert( - "test-ingester-4".into(), - IngesterPoolEntry { - client: IngesterServiceClient::mocked(), - status: IngesterStatus::Ready, - }, + assert_eq!(table.table.get(&key).unwrap().nodes.len(), 2); + + // Zero shards: node stays in table but becomes ineligible for routing. + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 0, + 0, ); - 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: 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: source_id.clone(), - shard_id: ShardId::from(1), - shard_state: ShardState::Open, - leader_id: "test-ingester-1".into(), - }, - RoutingEntry { - index_uid: index_uid.clone(), - 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: source_id.clone(), - shard_id: ShardId::from(3), - shard_state: ShardState::Open, - 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, vec![ShardId::from(0)]); + let entry = table.table.get(&key).unwrap(); + assert_eq!(entry.nodes.len(), 2); + assert_eq!(entry.nodes.get("node-1").unwrap().open_shard_count, 0); + assert_eq!(entry.nodes.get("node-1").unwrap().capacity_score, 0); + } - assert_eq!( - unavailable_leaders, - HashSet::from_iter([ - "test-ingester-1".into(), - "test-ingester-2".into(), - "test-ingester-3".into(), - ]) + #[test] + fn test_has_open_nodes() { + let mut table = RoutingTable::default(); + let pool = IngesterPool::default(); + + // Empty table. + assert!(!table.has_open_nodes("test-index", "test-source", &pool, &HashSet::new())); + + // Node exists but is not in pool. + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 8, + 3, + ); + assert!(!table.has_open_nodes("test-index", "test-source", &pool, &HashSet::new())); + + // Node is in pool → true. + pool.insert("node-1".into(), mocked_ingester(None)); + assert!(table.has_open_nodes("test-index", "test-source", &pool, &HashSet::new())); + + // Node is unavailable → false. + let unavailable: HashSet = HashSet::from(["node-1".into()]); + assert!(!table.has_open_nodes("test-index", "test-source", &pool, &unavailable)); + + // Second node available → true despite first being unavailable. + table.apply_capacity_update( + "node-2".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 6, + 2, ); + pool.insert("node-2".into(), mocked_ingester(None)); + assert!(table.has_open_nodes("test-index", "test-source", &pool, &unavailable)); + + // Node with capacity_score=0 is not eligible. + table.apply_capacity_update( + "node-2".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 0, + 2, + ); + assert!(!table.has_open_nodes("test-index", "test-source", &pool, &unavailable)); } #[test] - fn test_routing_table_entry_next_open_shard_round_robin() { - let index_uid = IndexUid::for_test("test-index", 0); - 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(); - - let error = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap_err(); - assert_eq!(error, NextOpenShardError::NoShardsAvailable); - - ingester_pool.insert( - "test-ingester-0".into(), - IngesterPoolEntry { - client: IngesterServiceClient::mocked(), - status: IngesterStatus::Ready, - }, + fn test_pick_node_prefers_same_az() { + let mut table = RoutingTable::new(Some("az-1".to_string())); + let pool = IngesterPool::default(); + + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 5, + 1, ); - ingester_pool.insert( - "test-ingester-1".into(), - IngesterPoolEntry { - client: IngesterServiceClient::mocked(), - status: IngesterStatus::Ready, - }, + table.apply_capacity_update( + "node-2".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 5, + 1, ); + pool.insert("node-1".into(), mocked_ingester(Some("az-1"))); + pool.insert("node-2".into(), mocked_ingester(Some("az-2"))); - 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: 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: 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: source_id.clone(), - shard_id: ShardId::from(3), - 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(), - }; - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(2)); - - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(3)); - - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(2)); - - 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: source_id.clone(), - shard_id: ShardId::from(1), - shard_state: ShardState::Closed, - leader_id: "test-ingester-0".into(), - }], - local_round_robin_idx: AtomicUsize::default(), - remote_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - 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: 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: 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: source_id.clone(), - shard_id: ShardId::from(5), - shard_state: ShardState::Open, - leader_id: "test-ingester-1".into(), - }, - ], - remote_round_robin_idx: AtomicUsize::default(), - }; - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(2)); - - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(5)); - - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) + let picked = table + .pick_node("test-index", "test-source", &pool, &HashSet::new()) .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(2)); - - rate_limited_shards.insert(ShardId::from(5)); - - let shard = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap(); - assert_eq!(shard.shard_id, ShardId::from(2)); + assert_eq!(picked.node_id, NodeId::from("node-1")); } #[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::from("test-source"); - - let ingester_pool = IngesterPool::default(); - ingester_pool.insert( - "test-ingester-0".into(), - IngesterPoolEntry { - client: IngesterServiceClient::mocked(), - status: IngesterStatus::Ready, - }, + fn test_pick_node_falls_back_to_cross_az() { + let mut table = RoutingTable::new(Some("az-1".to_string())); + let pool = IngesterPool::default(); + + table.apply_capacity_update( + "node-2".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 5, + 1, ); + pool.insert("node-2".into(), mocked_ingester(Some("az-2"))); - let rate_limited_shards = HashSet::from_iter([ShardId::from(1)]); - - 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: source_id.clone(), - shard_id: ShardId::from(1), - 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(), - }; - let error = table_entry - .next_open_shard_round_robin(&ingester_pool, &rate_limited_shards) - .unwrap_err(); - assert_eq!(error, NextOpenShardError::RateLimited); + let picked = table + .pick_node("test-index", "test-source", &pool, &HashSet::new()) + .unwrap(); + assert_eq!(picked.node_id, NodeId::from("node-2")); } #[test] - fn test_routing_table_entry_insert_open_shards() { - let index_uid_0 = IndexUid::for_test("test-index", 0); - 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(); - let remote_node_id: NodeId = "test-ingester-1".into(); - table_entry.insert_open_shards(&local_node_id, &local_node_id, &index_uid_0, &[]); - - assert_eq!(table_entry.local_shards.len(), 0); - assert_eq!(table_entry.remote_shards.len(), 0); - - table_entry.insert_open_shards( - &local_node_id, - &local_node_id, - &index_uid_0, - &[ShardId::from(2)], + fn test_pick_node_no_az_awareness() { + let mut table = RoutingTable::default(); + let pool = IngesterPool::default(); + + table.apply_capacity_update( + "node-1".into(), + IndexUid::for_test("test-index", 0), + "test-source".into(), + 5, + 1, ); + pool.insert("node-1".into(), mocked_ingester(Some("az-1"))); - assert_eq!(table_entry.local_shards.len(), 1); - assert_eq!(table_entry.remote_shards.len(), 0); - - assert_eq!(table_entry.local_shards[0].index_uid, index_uid_0); - assert_eq!(table_entry.local_shards[0].source_id, source_id); - assert_eq!(table_entry.local_shards[0].shard_id, ShardId::from(2)); - assert_eq!(table_entry.local_shards[0].shard_state, ShardState::Open); - assert_eq!(table_entry.local_shards[0].leader_id, local_node_id); + let picked = table + .pick_node("test-index", "test-source", &pool, &HashSet::new()) + .unwrap(); + assert_eq!(picked.node_id, NodeId::from("node-1")); + } - table_entry.local_shards[0].shard_state = ShardState::Closed; - table_entry.insert_open_shards( - &local_node_id, - &local_node_id, - &index_uid_0, - &[ShardId::from(1), ShardId::from(2)], + #[test] + fn test_pick_node_missing_entry() { + let table = RoutingTable::new(Some("az-1".to_string())); + let pool = IngesterPool::default(); + + assert!( + table + .pick_node("nonexistent", "source", &pool, &HashSet::new()) + .is_none() ); + } - assert_eq!(table_entry.local_shards.len(), 2); - assert_eq!(table_entry.remote_shards.len(), 0); - - assert_eq!(table_entry.local_shards[0].shard_id, ShardId::from(1)); - assert_eq!(table_entry.local_shards[0].shard_state, ShardState::Open); - assert_eq!(table_entry.local_shards[1].shard_id, ShardId::from(2)); - assert_eq!(table_entry.local_shards[1].shard_state, ShardState::Closed); - - table_entry.local_shards.clear(); - table_entry.insert_open_shards( - &local_node_id, - &remote_node_id, - &index_uid_0, - &[ShardId::from(2)], - ); + #[test] + fn test_power_of_two_choices() { + // 3 candidates: best appears in the random pair 2/3 of the time and always + // wins when it does, so it should win ~67% of 1000 runs. Asserting > 550 + // is ~7.5 standard deviations from the mean — effectively impossible to flake. + let high = IngesterNode { + node_id: "high".into(), + index_uid: IndexUid::for_test("idx", 0), + source_id: "src".into(), + capacity_score: 9, + open_shard_count: 2, + }; + let mid = IngesterNode { + node_id: "mid".into(), + index_uid: IndexUid::for_test("idx", 0), + source_id: "src".into(), + capacity_score: 5, + open_shard_count: 2, + }; + let low = IngesterNode { + node_id: "low".into(), + index_uid: IndexUid::for_test("idx", 0), + source_id: "src".into(), + capacity_score: 1, + open_shard_count: 2, + }; + let candidates: Vec<&IngesterNode> = vec![&high, &mid, &low]; - assert_eq!(table_entry.local_shards.len(), 0); - assert_eq!(table_entry.remote_shards.len(), 1); + let mut high_wins = 0; + for _ in 0..1000 { + if power_of_two_choices(&candidates).node_id == "high" { + high_wins += 1; + } + } + assert!(high_wins > 550, "high won only {high_wins}/1000 times"); + } - assert_eq!(table_entry.remote_shards[0].index_uid, index_uid_0); - assert_eq!(table_entry.remote_shards[0].source_id, source_id); - assert_eq!(table_entry.remote_shards[0].shard_id, ShardId::from(2)); - assert_eq!(table_entry.remote_shards[0].shard_state, ShardState::Open); - assert_eq!(table_entry.remote_shards[0].leader_id, remote_node_id); + #[test] + fn test_merge_from_shards() { + let mut table = RoutingTable::default(); + let index_uid = IndexUid::for_test("test-index", 0); + let key = ("test-index".to_string(), "test-source".to_string()); + + let make_shard = |id: u64, leader: &str, open: bool| Shard { + index_uid: Some(index_uid.clone()), + source_id: "test-source".to_string(), + shard_id: Some(ShardId::from(id)), + shard_state: if open { + ShardState::Open as i32 + } else { + ShardState::Closed as i32 + }, + leader_id: leader.to_string(), + ..Default::default() + }; - table_entry.remote_shards[0].shard_state = ShardState::Closed; - table_entry.insert_open_shards( - &local_node_id, - &remote_node_id, - &index_uid_0, - &[ShardId::from(1), ShardId::from(2)], - ); + // Two open shards on node-1, one open + one closed on node-2, only closed on node-3. + let shards = vec![ + make_shard(1, "node-1", true), + make_shard(2, "node-1", true), + make_shard(3, "node-2", true), + make_shard(4, "node-2", false), + make_shard(5, "node-3", false), + ]; + table.merge_from_shards(index_uid.clone(), "test-source".into(), shards); - assert_eq!(table_entry.local_shards.len(), 0); - assert_eq!(table_entry.remote_shards.len(), 2); + let entry = table.table.get(&key).unwrap(); + assert_eq!(entry.nodes.len(), 3); - assert_eq!(table_entry.remote_shards[0].shard_id, ShardId::from(1)); - assert_eq!(table_entry.remote_shards[0].shard_state, ShardState::Open); - assert_eq!(table_entry.remote_shards[1].shard_id, ShardId::from(2)); - assert_eq!(table_entry.remote_shards[1].shard_state, ShardState::Closed); + let n1 = entry.nodes.get("node-1").unwrap(); + assert_eq!(n1.open_shard_count, 2); + assert_eq!(n1.capacity_score, 5); - // Update index incarnation. - let index_uid_1 = IndexUid::for_test("test-index", 1); - table_entry.insert_open_shards( - &local_node_id, - &local_node_id, - &index_uid_1, - &[ShardId::from(1)], - ); + let n2 = entry.nodes.get("node-2").unwrap(); + assert_eq!(n2.open_shard_count, 1); - assert_eq!(table_entry.index_uid, index_uid_1); - assert_eq!(table_entry.local_shards.len(), 1); - assert_eq!(table_entry.remote_shards.len(), 0); + let n3 = entry.nodes.get("node-3").unwrap(); + assert_eq!(n3.open_shard_count, 0); - assert_eq!(table_entry.local_shards[0].index_uid, index_uid_1); - assert_eq!(table_entry.local_shards[0].source_id, source_id); - assert_eq!(table_entry.local_shards[0].shard_id, ShardId::from(1)); - assert_eq!(table_entry.local_shards[0].shard_state, ShardState::Open); - assert_eq!(table_entry.local_shards[0].leader_id, local_node_id); + // Merging again adds new nodes but preserves existing ones. + let shards = vec![make_shard(10, "node-4", true)]; + table.merge_from_shards(index_uid, "test-source".into(), shards); - // Ignore previous index incarnation. - table_entry.insert_open_shards( - &local_node_id, - &local_node_id, - &index_uid_0, - &[ShardId::from(12), ShardId::from(42), ShardId::from(1337)], - ); - assert_eq!(table_entry.index_uid, index_uid_1); - assert_eq!(table_entry.local_shards.len(), 1); - assert_eq!(table_entry.remote_shards.len(), 0); + let entry = table.table.get(&key).unwrap(); + assert_eq!(entry.nodes.len(), 4); + assert!(entry.nodes.contains_key("node-1")); + assert!(entry.nodes.contains_key("node-2")); + assert!(entry.nodes.contains_key("node-3")); + assert!(entry.nodes.contains_key("node-4")); } #[test] - fn test_routing_table_entry_close_shards() { - let index_uid = IndexUid::for_test("test-index", 0); - let source_id = SourceId::from("test-source"); + fn test_classify_az_locality() { + let table = RoutingTable::new(Some("az-1".to_string())); + let pool = IngesterPool::default(); + pool.insert("node-local".into(), mocked_ingester(Some("az-1"))); + pool.insert("node-remote".into(), mocked_ingester(Some("az-2"))); + pool.insert("node-no-az".into(), mocked_ingester(None)); - let mut table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); - table_entry.close_shards(&index_uid, &[]); - table_entry.close_shards(&index_uid, &[ShardId::from(1)]); - assert!(table_entry.local_shards.is_empty()); - assert!(table_entry.remote_shards.is_empty()); - - let mut table_entry = RoutingTableEntry { - index_uid: index_uid.clone(), - source_id: source_id.clone(), - local_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - 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: 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: source_id.clone(), - shard_id: ShardId::from(3), - shard_state: ShardState::Open, - leader_id: "test-ingester-0".into(), - }, - ], - local_round_robin_idx: AtomicUsize::default(), - remote_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - 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: 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: source_id.clone(), - shard_id: ShardId::from(7), - shard_state: ShardState::Open, - leader_id: "test-ingester-1".into(), - }, - ], - remote_round_robin_idx: AtomicUsize::default(), - }; - table_entry.close_shards( - &index_uid, - &[ - ShardId::from(1), - ShardId::from(3), - ShardId::from(4), - ShardId::from(6), - ShardId::from(8), - ], + assert_eq!( + table.classify_az_locality(&"node-local".into(), &pool), + "same_az" + ); + assert_eq!( + table.classify_az_locality(&"node-remote".into(), &pool), + "cross_az" + ); + assert_eq!( + table.classify_az_locality(&"node-no-az".into(), &pool), + "az_unaware" ); - assert!(table_entry.local_shards[0].shard_state.is_closed()); - assert!(table_entry.local_shards[1].shard_state.is_open()); - assert!(table_entry.local_shards[2].shard_state.is_closed()); - assert!(table_entry.remote_shards[0].shard_state.is_open()); - assert!(table_entry.remote_shards[1].shard_state.is_closed()); - assert!(table_entry.remote_shards[2].shard_state.is_open()); - } - - #[test] - fn test_routing_table_entry_delete_shards() { - let index_uid = IndexUid::for_test("test-index", 0); - 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, &[]); - table_entry.delete_shards(&index_uid, &[ShardId::from(1)]); - assert!(table_entry.local_shards.is_empty()); - assert!(table_entry.remote_shards.is_empty()); - let mut table_entry = RoutingTableEntry { - index_uid: index_uid.clone(), - source_id: source_id.clone(), - local_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - 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: 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: source_id.clone(), - shard_id: ShardId::from(3), - shard_state: ShardState::Open, - leader_id: "test-ingester-0".into(), - }, - ], - local_round_robin_idx: AtomicUsize::default(), - remote_shards: vec![ - RoutingEntry { - index_uid: index_uid.clone(), - 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: 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: source_id.clone(), - shard_id: ShardId::from(7), - shard_state: ShardState::Open, - leader_id: "test-ingester-1".into(), - }, - ], - remote_round_robin_idx: AtomicUsize::default(), - }; - table_entry.delete_shards( - &index_uid, - &[ - ShardId::from(1), - ShardId::from(3), - ShardId::from(4), - ShardId::from(6), - ShardId::from(8), - ], + let table_no_az = RoutingTable::default(); + assert_eq!( + table_no_az.classify_az_locality(&"node-local".into(), &pool), + "az_unaware" ); - assert_eq!(table_entry.local_shards.len(), 1); - assert_eq!(table_entry.local_shards[0].shard_id, ShardId::from(2)); - assert_eq!(table_entry.remote_shards.len(), 2); - assert_eq!(table_entry.remote_shards[0].shard_id, ShardId::from(5)); - assert_eq!(table_entry.remote_shards[1].shard_id, ShardId::from(7)); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/state.rs b/quickwit/quickwit-ingest/src/ingest_v2/state.rs index a19e4eb509d..bcf31b2f100 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/state.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/state.rs @@ -19,6 +19,8 @@ use std::path::Path; use std::sync::{Arc, Weak}; use std::time::{Duration, Instant}; +use bytesize::ByteSize; +use itertools::Itertools; use mrecordlog::error::{DeleteQueueError, TruncateError}; use quickwit_cluster::Cluster; use quickwit_common::pretty::PrettyDisplay; @@ -27,7 +29,7 @@ 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; -use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, ShardState}; +use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, ShardIds, ShardState}; use quickwit_proto::types::{DocMappingUid, IndexUid, Position, QueueId, SourceId, split_queue_id}; use tokio::sync::{Mutex, MutexGuard, RwLock, RwLockMappedWriteGuard, RwLockWriteGuard, watch}; use tracing::{error, info}; @@ -35,9 +37,10 @@ use tracing::{error, info}; use super::models::IngesterShard; use super::rate_meter::RateMeter; use super::replication::{ReplicationStreamTaskHandle, ReplicationTaskHandle}; +use super::wal_capacity_tracker::WalCapacityTracker; use crate::ingest_v2::mrecordlog_utils::{force_delete_queue, queue_position_range}; use crate::mrecordlog_async::MultiRecordLogAsync; -use crate::{FollowerId, LeaderId}; +use crate::{FollowerId, LeaderId, OpenShardCounts}; /// Stores the state of the ingester and attempts to prevent deadlocks by exposing an API that /// guarantees that the internal data structures are always locked in the same order. @@ -61,6 +64,7 @@ pub(super) struct InnerIngesterState { // Replication tasks running for each replication stream opened with leaders. pub replication_tasks: HashMap, cluster: Cluster, + pub wal_capacity_tracker: WalCapacityTracker, status: IngesterStatus, status_tx: watch::Sender, } @@ -93,10 +97,46 @@ impl InnerIngesterState { .max_by_key(|(available_permits, _)| *available_permits) .map(|(_, shard)| shard) } + + /// Returns per-source open shard counts and closed shard IDs for all advertisable, + /// non-replica shards. + pub fn get_shard_snapshot(&self) -> (OpenShardCounts, Vec) { + let grouped = self + .shards + .values() + .filter(|shard| shard.is_advertisable && !shard.is_replica()) + .map(|shard| ((shard.index_uid.clone(), shard.source_id.clone()), shard)) + .into_group_map(); + + let mut open_counts = Vec::new(); + let mut closed_shards = Vec::new(); + + for ((index_uid, source_id), shards) in grouped { + let mut open_count = 0; + let mut closed_ids = Vec::new(); + + for shard in shards { + if shard.is_open() { + open_count += 1; + } else if shard.is_closed() { + closed_ids.push(shard.shard_id.clone()); + } + } + open_counts.push((index_uid.clone(), source_id.clone(), open_count)); + if !closed_ids.is_empty() { + closed_shards.push(ShardIds { + index_uid: Some(index_uid), + source_id, + shard_ids: closed_ids, + }); + } + } + (open_counts, closed_shards) + } } impl IngesterState { - async fn create(cluster: Cluster) -> Self { + async fn create(cluster: Cluster, disk_capacity: ByteSize, memory_capacity: ByteSize) -> Self { let status = IngesterStatus::Initializing; let (status_tx, status_rx) = watch::channel(status); let mut inner = InnerIngesterState { @@ -105,6 +145,7 @@ impl IngesterState { replication_streams: Default::default(), replication_tasks: Default::default(), cluster, + wal_capacity_tracker: WalCapacityTracker::new(disk_capacity, memory_capacity), status, status_tx, }; @@ -125,9 +166,11 @@ impl IngesterState { pub async fn load( cluster: Cluster, wal_dir_path: &Path, + disk_capacity: ByteSize, + memory_capacity: ByteSize, rate_limiter_settings: RateLimiterSettings, ) -> Self { - let state = Self::create(cluster).await; + let state = Self::create(cluster, disk_capacity, memory_capacity).await; let state_clone = state.clone(); let wal_dir_path = wal_dir_path.to_path_buf(); @@ -141,9 +184,23 @@ impl IngesterState { #[cfg(test)] pub async fn for_test(cluster: Cluster) -> (tempfile::TempDir, Self) { + Self::for_test_with_disk_capacity(cluster, ByteSize::mb(256)).await + } + + #[cfg(test)] + pub async fn for_test_with_disk_capacity( + cluster: Cluster, + disk_capacity: ByteSize, + ) -> (tempfile::TempDir, Self) { let temp_dir = tempfile::tempdir().unwrap(); - let mut state = - IngesterState::load(cluster, temp_dir.path(), RateLimiterSettings::default()).await; + let mut state = IngesterState::load( + cluster, + temp_dir.path(), + disk_capacity, + ByteSize::mb(256), + RateLimiterSettings::default(), + ) + .await; state.wait_for_ready().await; @@ -154,8 +211,10 @@ impl IngesterState { /// queues. Empty queues are deleted, while non-empty queues are recovered. However, the /// corresponding shards are closed and become read-only. pub async fn init(&self, wal_dir_path: &Path, rate_limiter_settings: RateLimiterSettings) { - let mut inner_guard = self.inner.lock().await; + // Acquire locks in the same order as `lock_fully` (mrecordlog first, then inner) to + // prevent ABBA deadlocks with the broadcast capacity task. let mut mrecordlog_guard = self.mrecordlog.write().await; + let mut inner_guard = self.inner.lock().await; let now = Instant::now(); @@ -475,22 +534,26 @@ mod tests { use bytesize::ByteSize; use quickwit_cluster::{ChannelTransport, create_cluster_for_test}; use quickwit_config::service::QuickwitService; - use quickwit_proto::types::ShardId; + use quickwit_proto::types::{NodeId, ShardId, SourceId}; use tokio::time::timeout; use super::*; - #[tokio::test] - async fn test_ingester_state_does_not_lock_while_initializing() { - let cluster = create_cluster_for_test( + async fn test_cluster() -> Cluster { + create_cluster_for_test( Vec::new(), &[QuickwitService::Indexer.as_str()], &ChannelTransport::default(), true, ) .await - .unwrap(); - let state = IngesterState::create(cluster.clone()).await; + .unwrap() + } + + #[tokio::test] + async fn test_ingester_state_does_not_lock_while_initializing() { + let cluster = test_cluster().await; + let state = IngesterState::create(cluster, ByteSize::mb(256), ByteSize::mb(256)).await; let inner_guard = state.inner.lock().await; assert_eq!(inner_guard.status(), IngesterStatus::Initializing); @@ -505,15 +568,8 @@ mod tests { #[tokio::test] async fn test_ingester_state_failed() { - 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 cluster = test_cluster().await; + let state = IngesterState::create(cluster, ByteSize::mb(256), ByteSize::mb(256)).await; state .inner @@ -531,15 +587,8 @@ mod tests { #[tokio::test] async fn test_ingester_state_init() { - 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 cluster = test_cluster().await; + let mut state = IngesterState::create(cluster, ByteSize::mb(256), ByteSize::mb(256)).await; let temp_dir = tempfile::tempdir().unwrap(); state @@ -706,15 +755,9 @@ mod tests { #[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 cluster = test_cluster().await; + let state = + IngesterState::create(cluster.clone(), ByteSize::mb(256), ByteSize::mb(256)).await; let temp_dir = tempfile::tempdir().unwrap(); state @@ -733,4 +776,88 @@ mod tests { let status = IngesterStatus::from_json_str_name(&status_json_str).unwrap(); assert_eq!(status, IngesterStatus::Failed); } + + fn open_shard( + index_uid: IndexUid, + source_id: SourceId, + shard_id: ShardId, + is_replica: bool, + ) -> IngesterShard { + let builder = if is_replica { + IngesterShard::new_replica(index_uid, source_id, shard_id, NodeId::from("test-leader")) + } else { + IngesterShard::new_solo(index_uid, source_id, shard_id) + }; + builder.advertisable().build() + } + + #[tokio::test] + async fn test_get_shard_snapshot() { + let cluster = test_cluster().await; + 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); + + // source-a: 2 open shards + 1 closed shard + 1 replica (ignored). + let s = open_shard( + index_uid.clone(), + "source-a".into(), + ShardId::from(1), + false, + ); + state_guard.shards.insert(s.queue_id(), s); + let s = open_shard( + index_uid.clone(), + "source-a".into(), + ShardId::from(2), + false, + ); + state_guard.shards.insert(s.queue_id(), s); + let s = IngesterShard::new_solo(index_uid.clone(), "source-a".into(), ShardId::from(3)) + .with_state(ShardState::Closed) + .advertisable() + .build(); + state_guard.shards.insert(s.queue_id(), s); + let s = open_shard(index_uid.clone(), "source-a".into(), ShardId::from(4), true); + state_guard.shards.insert(s.queue_id(), s); + + // source-b: 2 closed shards, no open shards. + let s = IngesterShard::new_solo(index_uid.clone(), "source-b".into(), ShardId::from(5)) + .with_state(ShardState::Closed) + .advertisable() + .build(); + state_guard.shards.insert(s.queue_id(), s); + let s = IngesterShard::new_solo(index_uid.clone(), "source-b".into(), ShardId::from(6)) + .with_state(ShardState::Closed) + .advertisable() + .build(); + state_guard.shards.insert(s.queue_id(), s); + + let (mut open_counts, mut closed_shards) = state_guard.get_shard_snapshot(); + + // Open counts: source-a has 2, source-b has 0. + open_counts.sort_by(|a, b| a.1.cmp(&b.1)); + assert_eq!(open_counts.len(), 2); + assert_eq!( + open_counts[0], + (index_uid.clone(), SourceId::from("source-a"), 2) + ); + assert_eq!( + open_counts[1], + (index_uid.clone(), SourceId::from("source-b"), 0) + ); + + // Closed shards: source-a has shard 3, source-b has shards 5 and 6. + closed_shards.sort_by(|a, b| a.source_id.cmp(&b.source_id)); + assert_eq!(closed_shards.len(), 2); + + assert_eq!(closed_shards[0].source_id, "source-a"); + assert_eq!(closed_shards[0].shard_ids, vec![ShardId::from(3)]); + + assert_eq!(closed_shards[1].source_id, "source-b"); + let mut source_b_ids = closed_shards[1].shard_ids.clone(); + source_b_ids.sort(); + assert_eq!(source_b_ids, vec![ShardId::from(5), ShardId::from(6)]); + } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/wal_capacity_tracker.rs b/quickwit/quickwit-ingest/src/ingest_v2/wal_capacity_tracker.rs new file mode 100644 index 00000000000..f24e8254053 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/wal_capacity_tracker.rs @@ -0,0 +1,249 @@ +// 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 bytesize::ByteSize; +use quickwit_common::ring_buffer::RingBuffer; + +/// The lookback window length is meant to capture readings far enough back in time to give +/// a rough rate of change estimate. At size 6, with broadcast interval of 5 seconds, this would be +/// 30 seconds of readings. +const WAL_CAPACITY_LOOKBACK_WINDOW_LEN: usize = 6; + +/// The ring buffer stores one extra element so that `delta()` can compare the newest reading +/// with the one that is exactly `WAL_CAPACITY_LOOKBACK_WINDOW_LEN` steps ago. Otherwise, that +/// reading would be discarded when the next reading is inserted. +const WAL_CAPACITY_READINGS_LEN: usize = WAL_CAPACITY_LOOKBACK_WINDOW_LEN + 1; + +struct WalCapacityTimeSeries { + capacity: ByteSize, + readings: RingBuffer, +} + +impl WalCapacityTimeSeries { + fn new(capacity: ByteSize) -> Self { + #[cfg(not(test))] + assert!(capacity.as_u64() > 0); + Self { + capacity, + readings: RingBuffer::default(), + } + } + + fn record_and_score(&mut self, used: ByteSize) -> usize { + self.record(used); + let remaining = self.current().unwrap_or(1.0); + let delta = self.delta().unwrap_or(0.0); + compute_capacity_score(remaining, delta) + } + + fn score(&self, used: ByteSize) -> usize { + let remaining = 1.0 - (used.as_u64() as f64 / self.capacity.as_u64() as f64); + let delta = self.delta().unwrap_or(0.0); + compute_capacity_score(remaining, delta) + } + + fn record(&mut self, used: ByteSize) { + let remaining = 1.0 - (used.as_u64() as f64 / self.capacity.as_u64() as f64); + self.readings.push_back(remaining.clamp(0.0, 1.0)); + } + + fn current(&self) -> Option { + self.readings.last() + } + + fn delta(&self) -> Option { + let current = self.readings.last()?; + let oldest = self.readings.front()?; + Some(current - oldest) + } +} + +pub struct WalCapacityTracker { + disk: WalCapacityTimeSeries, + memory: WalCapacityTimeSeries, +} + +impl WalCapacityTracker { + pub fn new(disk_capacity: ByteSize, memory_capacity: ByteSize) -> Self { + Self { + disk: WalCapacityTimeSeries::new(disk_capacity), + memory: WalCapacityTimeSeries::new(memory_capacity), + } + } + + /// Records disk and memory usage readings and returns the resulting capacity score. + /// The score is the minimum of the individual disk and memory scores. + pub fn record_and_score(&mut self, disk_used: ByteSize, memory_used: ByteSize) -> usize { + let disk_score = self.disk.record_and_score(disk_used); + let memory_score = self.memory.record_and_score(memory_used); + disk_score.min(memory_score) + } + + /// Computes a capacity score for the given usage without recording it. + pub fn score(&self, disk_used: ByteSize, memory_used: ByteSize) -> usize { + let disk_score = self.disk.score(disk_used); + let memory_score = self.memory.score(memory_used); + disk_score.min(memory_score) + } +} + +/// Computes a capacity score from 0 to 10 using a PD controller. +/// +/// The score has two components: +/// +/// - **P (proportional):** How much WAL capacity remains right now. An ingester with 100% free +/// capacity gets `PROPORTIONAL_WEIGHT` points; 50% gets half; and so on. If remaining capacity +/// drops to `MIN_PERMISSIBLE_CAPACITY` or below, the score is immediately 0. +/// +/// - **D (derivative):** Up to `DERIVATIVE_WEIGHT` bonus points based on how fast remaining +/// capacity is changing over the lookback window. A higher drain rate is worse, so we invert it: +/// `drain / MAX_DRAIN_RATE` normalizes the drain to a 0–1 penalty, and subtracting from 1 +/// converts it into a 0–1 bonus. Multiplied by `DERIVATIVE_WEIGHT`, a stable node gets the full +/// bonus and a node draining at `MAX_DRAIN_RATE` or faster gets nothing. +/// +/// Putting it together: a completely idle ingester scores 10 (8 + 2). +/// One that is full but stable scores ~2. One that is draining rapidly scores less. +/// A score of 0 means the ingester is at or below minimum permissible capacity. +/// +/// Below this remaining capacity fraction, the score is immediately 0. +const MIN_PERMISSIBLE_CAPACITY: f64 = 0.05; +/// Weight of the proportional term (max points from P). +const PROPORTIONAL_WEIGHT: f64 = 8.0; +/// Weight of the derivative term (max points from D). +const DERIVATIVE_WEIGHT: f64 = 2.0; +/// The drain rate (as a fraction of total capacity over the lookback window) at which the +/// derivative penalty is fully applied. Drain rates beyond this are clamped. +const MAX_DRAIN_RATE: f64 = 0.10; + +fn compute_capacity_score(remaining_capacity: f64, capacity_delta: f64) -> usize { + if remaining_capacity <= MIN_PERMISSIBLE_CAPACITY { + return 0; + } + let p = PROPORTIONAL_WEIGHT * remaining_capacity; + let drain = (-capacity_delta).clamp(0.0, MAX_DRAIN_RATE); + let d = DERIVATIVE_WEIGHT * (1.0 - drain / MAX_DRAIN_RATE); + (p + d).clamp(0.0, 10.0) as usize +} + +#[cfg(test)] +mod tests { + use super::*; + + fn ts() -> WalCapacityTimeSeries { + WalCapacityTimeSeries::new(ByteSize::b(100)) + } + + /// Helper: record a reading with `used` bytes against the series' fixed capacity. + fn record(series: &mut WalCapacityTimeSeries, used: u64) { + series.record(ByteSize::b(used)); + } + + #[test] + fn test_wal_disk_capacity_current_after_record() { + let mut series = WalCapacityTimeSeries::new(ByteSize::b(256)); + // 192 of 256 used => 25% remaining + series.record(ByteSize::b(192)); + assert_eq!(series.current(), Some(0.25)); + + // 16 of 256 used => 93.75% remaining + series.record(ByteSize::b(16)); + assert_eq!(series.current(), Some(0.9375)); + } + + #[test] + fn test_wal_disk_capacity_record_saturates_at_zero() { + let mut series = ts(); + // 200 used out of 100 capacity => clamped to 0.0 + record(&mut series, 200); + assert_eq!(series.current(), Some(0.0)); + } + + #[test] + fn test_wal_disk_capacity_delta_growing() { + let mut series = ts(); + // oldest: 60 of 100 used => 40% remaining + record(&mut series, 60); + // current: 20 of 100 used => 80% remaining + record(&mut series, 20); + // delta = 0.80 - 0.40 = 0.40 + assert_eq!(series.delta(), Some(0.40)); + } + + #[test] + fn test_wal_disk_capacity_delta_shrinking() { + let mut series = ts(); + // oldest: 20 of 100 used => 80% remaining + record(&mut series, 20); + // current: 60 of 100 used => 40% remaining + record(&mut series, 60); + // delta = 0.40 - 0.80 = -0.40 + assert_eq!(series.delta(), Some(-0.40)); + } + + #[test] + fn test_capacity_score_draining_vs_stable() { + // Node A: capacity draining — usage increases 10, 20, ..., 70 over 7 ticks. + let mut node_a = ts(); + for used in (10..=70).step_by(10) { + record(&mut node_a, used); + } + let a_remaining = node_a.current().unwrap(); + let a_delta = node_a.delta().unwrap(); + let a_score = compute_capacity_score(a_remaining, a_delta); + + // Node B: steady at 50% usage over 7 ticks. + let mut node_b = ts(); + for _ in 0..7 { + record(&mut node_b, 50); + } + let b_remaining = node_b.current().unwrap(); + let b_delta = node_b.delta().unwrap(); + let b_score = compute_capacity_score(b_remaining, b_delta); + + // p=2.4, d=0 (max drain) => 2 + assert_eq!(a_score, 2); + // p=4, d=2 (stable) => 6 + assert_eq!(b_score, 6); + assert!(b_score > a_score); + } + + #[test] + fn test_wal_disk_capacity_delta_spans_lookback_window() { + let mut series = ts(); + + // Fill to exactly the lookback window length (6 readings), all same value. + for _ in 0..WAL_CAPACITY_LOOKBACK_WINDOW_LEN { + record(&mut series, 50); + } + assert_eq!(series.delta(), Some(0.0)); + + // 7th reading fills the ring buffer. Delta spans 6 intervals. + record(&mut series, 0); + assert_eq!(series.delta(), Some(0.50)); + + // 8th reading evicts the oldest 50-remaining. Delta still spans 6 intervals. + record(&mut series, 0); + assert_eq!(series.delta(), Some(0.50)); + } + + #[test] + fn test_wal_capacity_tracker_returns_min() { + let mut tracker = WalCapacityTracker::new(ByteSize::b(100), ByteSize::b(100)); + // Disk 10% used (score 9), memory 90% used (score 2) → returns 2. + assert_eq!( + tracker.record_and_score(ByteSize::b(10), ByteSize::b(90)), + 2 + ); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 311aef138d8..50f92654efb 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -23,8 +23,8 @@ use quickwit_proto::ingest::ingester::{PersistFailure, PersistFailureReason, Per use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSubrequest, IngestSuccess, }; -use quickwit_proto::ingest::{IngestV2Error, RateLimitingCause}; -use quickwit_proto::types::{NodeId, ShardId, SubrequestId}; +use quickwit_proto::ingest::{IngestV2Error, RateLimitingCause, ShardIds}; +use quickwit_proto::types::{NodeId, SubrequestId}; use tracing::warn; use super::publish_tracker::PublishTracker; @@ -35,7 +35,6 @@ use super::router::PersistRequestSummary; #[derive(Default)] pub(super) struct IngestWorkbench { pub subworkbenches: BTreeMap, - pub rate_limited_shards: HashSet, pub num_successes: usize, /// The number of batch persist attempts. This is not sum of the number of attempts for each /// subrequest. @@ -48,6 +47,7 @@ pub(super) struct IngestWorkbench { /// (The point here is to make sure we do not wait for the failure detection to kick the node /// out of the ingest node.) pub unavailable_leaders: HashSet, + pub closed_shards: Vec, publish_tracker: Option, } @@ -228,13 +228,6 @@ impl IngestWorkbench { self.record_failure(subrequest_id, SubworkbenchFailure::NoShardsAvailable); } - pub fn record_rate_limited(&mut self, subrequest_id: SubrequestId) { - self.record_failure( - subrequest_id, - SubworkbenchFailure::RateLimited(RateLimitingCause::ShardRateLimiting), - ); - } - /// Marks a node as unavailable for the span of the workbench. /// /// Remaining attempts will treat the node as if it was not in the ingester pool. @@ -433,7 +426,7 @@ mod tests { assert!(!subworkbench.last_failure_is_transient()); subworkbench.last_failure_opt = Some(SubworkbenchFailure::Persist( - PersistFailureReason::ShardRateLimited, + PersistFailureReason::NoShardsAvailable, )); assert!(subworkbench.is_pending()); assert!(subworkbench.last_failure_is_transient()); @@ -567,7 +560,6 @@ mod tests { let persist_failure = PersistFailure { subrequest_id: 1, - shard_id: Some(shard_id_2.clone()), ..Default::default() }; workbench.record_persist_failure(&persist_failure); @@ -807,14 +799,13 @@ mod tests { let persist_failure = PersistFailure { subrequest_id: 42, - reason: PersistFailureReason::ShardRateLimited as i32, + reason: PersistFailureReason::NoShardsAvailable as i32, ..Default::default() }; workbench.record_persist_failure(&persist_failure); let persist_failure = PersistFailure { subrequest_id: 0, - shard_id: Some(ShardId::from(1)), reason: PersistFailureReason::WalFull as i32, ..Default::default() }; 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 9c3de3a6b85..e998cff0b3c 100644 --- a/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/ingest_v2_tests.rs @@ -927,6 +927,7 @@ async fn test_graceful_shutdown_no_data_loss() { // Add a second indexer after the shard has been created on the first one. sandbox.add_node([QuickwitService::Indexer]).await; + sandbox.wait_for_cluster_num_ready_nodes(3).await.unwrap(); // Remove the first indexer (the shard owner) from the sandbox and get its // shutdown handle. After this call, rest_client(Indexer) returns the @@ -995,7 +996,7 @@ async fn test_graceful_shutdown_no_data_loss() { .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()) + tokio::time::timeout(Duration::from_secs(30), sandbox.shutdown()) .await .unwrap() .unwrap(); diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index f7341b12862..d48bc553a62 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -73,14 +73,27 @@ message PersistSubrequest { uint32 subrequest_id = 1; quickwit.common.IndexUid index_uid = 2; string source_id = 3; - quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.DocBatchV2 doc_batch = 5; + reserved 4; } message PersistResponse { string leader_id = 1; repeated PersistSuccess successes = 2; repeated PersistFailure failures = 3; + RoutingUpdate routing_update = 4; +} + +message RoutingUpdate { + uint32 capacity_score = 1; + repeated SourceShardUpdate source_shard_updates = 2; + repeated quickwit.ingest.ShardIds closed_shards = 3; +} + +message SourceShardUpdate { + quickwit.common.IndexUid index_uid = 1; + string source_id = 2; + uint32 open_shard_count = 3; } message PersistSuccess { @@ -96,19 +109,18 @@ message PersistSuccess { enum PersistFailureReason { PERSIST_FAILURE_REASON_UNSPECIFIED = 0; - PERSIST_FAILURE_REASON_SHARD_NOT_FOUND = 1; - PERSIST_FAILURE_REASON_SHARD_CLOSED = 2; - PERSIST_FAILURE_REASON_SHARD_RATE_LIMITED = 3; PERSIST_FAILURE_REASON_WAL_FULL = 4; PERSIST_FAILURE_REASON_TIMEOUT = 5; + PERSIST_FAILURE_REASON_NO_SHARDS_AVAILABLE = 6; + PERSIST_FAILURE_REASON_NODE_UNAVAILABLE = 7; } message PersistFailure { uint32 subrequest_id = 1; quickwit.common.IndexUid index_uid = 2; string source_id = 3; - quickwit.ingest.ShardId shard_id = 4; PersistFailureReason reason = 5; + reserved 4; } message SynReplicationMessage { 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 ae434b45529..c472e2ea3ec 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -37,8 +37,6 @@ pub struct PersistSubrequest { pub index_uid: ::core::option::Option, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(message, optional, tag = "4")] - pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub doc_batch: ::core::option::Option, } @@ -51,6 +49,28 @@ pub struct PersistResponse { pub successes: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "3")] pub failures: ::prost::alloc::vec::Vec, + #[prost(message, optional, tag = "4")] + pub routing_update: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct RoutingUpdate { + #[prost(uint32, tag = "1")] + pub capacity_score: u32, + #[prost(message, repeated, tag = "2")] + pub source_shard_updates: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub closed_shards: ::prost::alloc::vec::Vec, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] +pub struct SourceShardUpdate { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(uint32, tag = "3")] + pub open_shard_count: u32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -79,8 +99,6 @@ pub struct PersistFailure { pub index_uid: ::core::option::Option, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(message, optional, tag = "4")] - pub shard_id: ::core::option::Option, #[prost(enumeration = "PersistFailureReason", tag = "5")] pub reason: i32, } @@ -397,11 +415,10 @@ pub struct ObservationMessage { #[repr(i32)] pub enum PersistFailureReason { Unspecified = 0, - ShardNotFound = 1, - ShardClosed = 2, - ShardRateLimited = 3, WalFull = 4, Timeout = 5, + NoShardsAvailable = 6, + NodeUnavailable = 7, } impl PersistFailureReason { /// String value of the enum field names used in the ProtoBuf definition. @@ -411,22 +428,20 @@ impl PersistFailureReason { pub fn as_str_name(&self) -> &'static str { match self { Self::Unspecified => "PERSIST_FAILURE_REASON_UNSPECIFIED", - Self::ShardNotFound => "PERSIST_FAILURE_REASON_SHARD_NOT_FOUND", - Self::ShardClosed => "PERSIST_FAILURE_REASON_SHARD_CLOSED", - Self::ShardRateLimited => "PERSIST_FAILURE_REASON_SHARD_RATE_LIMITED", Self::WalFull => "PERSIST_FAILURE_REASON_WAL_FULL", Self::Timeout => "PERSIST_FAILURE_REASON_TIMEOUT", + Self::NoShardsAvailable => "PERSIST_FAILURE_REASON_NO_SHARDS_AVAILABLE", + Self::NodeUnavailable => "PERSIST_FAILURE_REASON_NODE_UNAVAILABLE", } } /// Creates an enum from field names used in the ProtoBuf definition. pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "PERSIST_FAILURE_REASON_UNSPECIFIED" => Some(Self::Unspecified), - "PERSIST_FAILURE_REASON_SHARD_NOT_FOUND" => Some(Self::ShardNotFound), - "PERSIST_FAILURE_REASON_SHARD_CLOSED" => Some(Self::ShardClosed), - "PERSIST_FAILURE_REASON_SHARD_RATE_LIMITED" => Some(Self::ShardRateLimited), "PERSIST_FAILURE_REASON_WAL_FULL" => Some(Self::WalFull), "PERSIST_FAILURE_REASON_TIMEOUT" => Some(Self::Timeout), + "PERSIST_FAILURE_REASON_NO_SHARDS_AVAILABLE" => Some(Self::NoShardsAvailable), + "PERSIST_FAILURE_REASON_NODE_UNAVAILABLE" => Some(Self::NodeUnavailable), _ => None, } } @@ -470,9 +485,8 @@ impl ReplicateFailureReason { #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] #[repr(i32)] pub enum IngesterStatus { - /// For nodes without an ingester. Unspecified = 0, - /// The ingester is live but not ready yet to accept requests (i.e. the Write-Ahead Log is not loaded yet). + /// The ingester is live but not ready yet to accept requests. Initializing = 1, /// The ingester is ready and accepts read and write requests. Ready = 2, diff --git a/quickwit/quickwit-proto/src/getters.rs b/quickwit/quickwit-proto/src/getters.rs index a327c1717a7..4d8e6f058e6 100644 --- a/quickwit/quickwit-proto/src/getters.rs +++ b/quickwit/quickwit-proto/src/getters.rs @@ -111,6 +111,7 @@ generate_getters! { ShardIds, ShardPKey, TruncateShardsSubrequest, + SourceShardUpdate, // Metastore API AcquireShardsRequest, @@ -209,8 +210,6 @@ generate_getters! { InitShardFailure, OpenFetchStreamRequest, OpenShardSubrequest, - PersistFailure, - PersistSubrequest, PersistSuccess, ReplicateFailure, ReplicateSubrequest, diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs index 0236380f8f5..b7308d05d05 100644 --- a/quickwit/quickwit-proto/src/ingest/ingester.rs +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -113,12 +113,6 @@ impl OpenFetchStreamRequest { } } -impl PersistSubrequest { - pub fn queue_id(&self) -> QueueId { - queue_id(self.index_uid(), &self.source_id, self.shard_id()) - } -} - impl PersistSuccess { pub fn queue_id(&self) -> QueueId { queue_id(self.index_uid(), &self.source_id, self.shard_id()) diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index fda347d7931..6d529d79fbd 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -313,11 +313,10 @@ impl From for IngestFailureReason { fn from(reason: PersistFailureReason) -> Self { match reason { PersistFailureReason::Unspecified => IngestFailureReason::Unspecified, - PersistFailureReason::ShardNotFound => IngestFailureReason::NoShardsAvailable, - PersistFailureReason::ShardClosed => IngestFailureReason::NoShardsAvailable, + PersistFailureReason::NoShardsAvailable => IngestFailureReason::NoShardsAvailable, PersistFailureReason::WalFull => IngestFailureReason::WalFull, - PersistFailureReason::ShardRateLimited => IngestFailureReason::ShardRateLimited, PersistFailureReason::Timeout => IngestFailureReason::Timeout, + PersistFailureReason::NodeUnavailable => IngestFailureReason::NoShardsAvailable, } } } @@ -326,8 +325,8 @@ impl From for PersistFailureReason { fn from(reason: ReplicateFailureReason) -> Self { match reason { ReplicateFailureReason::Unspecified => PersistFailureReason::Unspecified, - ReplicateFailureReason::ShardNotFound => PersistFailureReason::ShardNotFound, - ReplicateFailureReason::ShardClosed => PersistFailureReason::ShardClosed, + ReplicateFailureReason::ShardNotFound => PersistFailureReason::NoShardsAvailable, + ReplicateFailureReason::ShardClosed => PersistFailureReason::NoShardsAvailable, ReplicateFailureReason::WalFull => PersistFailureReason::WalFull, } } diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 1137decf466..3ad5bd72aa7 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -83,8 +83,9 @@ use quickwit_indexing::start_indexing_service; use quickwit_ingest::{ GetMemoryCapacity, IngestRequest, IngestRouter, IngestServiceClient, Ingester, IngesterPool, 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, + setup_ingester_capacity_update_listener, 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}; @@ -934,8 +935,12 @@ async fn setup_ingest_v2( ingester_pool.clone(), replication_factor, event_broker.clone(), + node_config.availability_zone.clone(), ); ingest_router.subscribe(); + setup_ingester_capacity_update_listener(cluster.clone(), event_broker.clone()) + .await + .forever(); let ingest_router_service = IngestRouterServiceClient::tower() .stack_layer(INGEST_GRPC_SERVER_METRICS_LAYER.clone()) @@ -1008,12 +1013,12 @@ fn setup_ingester_pool( ); Some(change) } + // only update the ingester pool when the ingester status changes, to avoid + // unnecessary churn ClusterChange::Update { previous, updated } if updated.is_indexer() && previous.ingester_status() != updated.ingester_status() => { - // only update the ingester pool when the ingester status changes, to avoid - // unnecessary churn let change = build_ingester_insert_change( &updated, ingester_opt_clone, @@ -1057,6 +1062,7 @@ fn build_ingester_insert_change( let pool_entry = IngesterPoolEntry { client: ingester_service, status: node.ingester_status(), + availability_zone: node.availability_zone().map(|az| az.to_string()), }; Change::Insert(node_id, pool_entry) } diff --git a/quickwit/rest-api-tests/scenarii/tag_fields/0002_negative_tags.yaml b/quickwit/rest-api-tests/scenarii/tag_fields/0002_negative_tags.yaml index f1f900c342c..99f0e5ed285 100644 --- a/quickwit/rest-api-tests/scenarii/tag_fields/0002_negative_tags.yaml +++ b/quickwit/rest-api-tests/scenarii/tag_fields/0002_negative_tags.yaml @@ -1,23 +1,23 @@ # regression test for https://github.com/quickwit-oss/quickwit/issues/4698 -endpoint: simple/search +endpoint: tag-simple/search params: query: "tag:1" expected: num_hits: 3 --- -endpoint: simple/search +endpoint: tag-simple/search params: query: "-tag:2" expected: num_hits: 4 --- -endpoint: simple/search +endpoint: tag-simple/search params: query: "tag:2" expected: num_hits: 1 --- -endpoint: simple/search +endpoint: tag-simple/search params: query: "-tag:1" expected: diff --git a/quickwit/rest-api-tests/scenarii/tag_fields/_setup.quickwit.yaml b/quickwit/rest-api-tests/scenarii/tag_fields/_setup.quickwit.yaml index 1208ca48343..4ae0b2eb465 100644 --- a/quickwit/rest-api-tests/scenarii/tag_fields/_setup.quickwit.yaml +++ b/quickwit/rest-api-tests/scenarii/tag_fields/_setup.quickwit.yaml @@ -4,14 +4,14 @@ endpoint: indexes/allowedtypes status_code: null --- method: DELETE -endpoint: indexes/simple +endpoint: indexes/tag-simple status_code: null --- method: POST endpoint: indexes/ json: version: "0.7" - index_id: simple + index_id: tag-simple doc_mapping: field_mappings: - name: seq @@ -21,7 +21,7 @@ json: tag_fields: ["tag"] --- method: POST -endpoint: simple/ingest +endpoint: tag-simple/ingest params: commit: force ndjson: @@ -29,7 +29,7 @@ ndjson: - {"seq": 2, "tag": 2} --- method: POST -endpoint: simple/ingest +endpoint: tag-simple/ingest params: commit: force ndjson: @@ -37,7 +37,7 @@ ndjson: - {"seq": 3, "tag": null} --- method: POST -endpoint: simple/ingest +endpoint: tag-simple/ingest params: commit: force ndjson: diff --git a/quickwit/rest-api-tests/scenarii/tag_fields/_teardown.quickwit.yaml b/quickwit/rest-api-tests/scenarii/tag_fields/_teardown.quickwit.yaml index fa0ca391b51..0c3ac8fd6a4 100644 --- a/quickwit/rest-api-tests/scenarii/tag_fields/_teardown.quickwit.yaml +++ b/quickwit/rest-api-tests/scenarii/tag_fields/_teardown.quickwit.yaml @@ -3,4 +3,4 @@ endpoint: indexes/allowedtypes status_code: null --- method: DELETE -endpoint: indexes/simple +endpoint: indexes/tag-simple