From 9d0a223cdbc44a3689bf5cb9785c96e441e9c565 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Tue, 16 Jan 2024 12:59:30 -0500 Subject: [PATCH] Replace `shard_id: u64` with generic `shard_id: ShardId(ByteString)` (#4379) --- quickwit/clippy.toml | 7 + quickwit/quickwit-cluster/src/cluster.rs | 63 +++--- .../src/control_plane.rs | 47 ++-- .../src/indexing_plan.rs | 14 +- .../src/indexing_scheduler/mod.rs | 7 +- .../src/indexing_scheduler/scheduling/mod.rs | 200 ++++++++++++----- .../src/ingest/ingest_controller.rs | 182 ++++++++-------- .../quickwit-control-plane/src/model/mod.rs | 28 +-- .../src/model/shard_table.rs | 169 ++++++--------- .../src/actors/indexing_service.rs | 4 +- .../src/models/shard_positions.rs | 64 ++++-- .../src/source/ingest/mod.rs | 169 ++++++++------- .../src/ingest_v2/broadcast.rs | 21 +- .../quickwit-ingest/src/ingest_v2/fetch.rs | 106 ++++----- .../quickwit-ingest/src/ingest_v2/ingester.rs | 189 ++++++++-------- .../src/ingest_v2/replication.rs | 78 +++---- .../quickwit-ingest/src/ingest_v2/router.rs | 87 ++++---- .../src/ingest_v2/routing_table.rs | 204 +++++++++++------- .../src/ingest_v2/workbench.rs | 3 +- quickwit/quickwit-metastore/src/checkpoint.rs | 4 + .../file_backed_index/mod.rs | 16 +- .../file_backed_index/serialize.rs | 26 ++- .../file_backed_index/shards.rs | 144 +++++-------- .../quickwit-metastore/src/tests/shard.rs | 38 +--- .../file-backed-index/v0.6.expected.json | 15 -- .../test-data/file-backed-index/v0.6.json | 15 -- .../file-backed-index/v0.7.expected.json | 24 +-- .../test-data/file-backed-index/v0.7.json | 24 +-- quickwit/quickwit-proto/build.rs | 13 +- .../protos/quickwit/indexing.proto | 4 +- .../protos/quickwit/ingest.proto | 12 +- .../protos/quickwit/ingester.proto | 22 +- .../protos/quickwit/metastore.proto | 12 +- .../protos/quickwit/router.proto | 2 +- .../src/codegen/quickwit/quickwit.indexing.rs | 4 +- .../quickwit/quickwit.ingest.ingester.rs | 44 ++-- .../quickwit/quickwit.ingest.router.rs | 4 +- .../src/codegen/quickwit/quickwit.ingest.rs | 8 +- .../codegen/quickwit/quickwit.metastore.rs | 20 +- quickwit/quickwit-proto/src/indexing/mod.rs | 2 +- .../quickwit-proto/src/ingest/ingester.rs | 128 ++++++++--- quickwit/quickwit-proto/src/ingest/mod.rs | 16 +- quickwit/quickwit-proto/src/metastore/mod.rs | 14 +- quickwit/quickwit-proto/src/types/mod.rs | 27 +-- quickwit/quickwit-proto/src/types/position.rs | 7 + quickwit/quickwit-proto/src/types/shard_id.rs | 168 +++++++++++++++ .../src/elastic_search_api/bulk_v2.rs | 8 +- 47 files changed, 1393 insertions(+), 1070 deletions(-) create mode 100644 quickwit/quickwit-proto/src/types/shard_id.rs diff --git a/quickwit/clippy.toml b/quickwit/clippy.toml index 01ab61466e1..06bfa7456e3 100644 --- a/quickwit/clippy.toml +++ b/quickwit/clippy.toml @@ -1,3 +1,10 @@ disallowed-methods = [ "std::path::Path::exists" ] +ignore-interior-mutability = [ + "bytes::Bytes", + "bytestring::ByteString", + "quickwit_ingest::ShardInfo", + "quickwit_ingest::ShardInfos", + "quickwit_proto::types::ShardId", +] diff --git a/quickwit/quickwit-cluster/src/cluster.rs b/quickwit/quickwit-cluster/src/cluster.rs index bf12a3eb726..76f0d64baa0 100644 --- a/quickwit/quickwit-cluster/src/cluster.rs +++ b/quickwit/quickwit-cluster/src/cluster.rs @@ -467,27 +467,23 @@ pub(crate) fn set_indexing_tasks_in_node_state( fn indexing_task_to_chitchat_kv(indexing_task: &IndexingTask) -> (String, String) { let IndexingTask { - pipeline_uid: _, index_uid, source_id, shard_ids, + pipeline_uid: _, } = indexing_task; let key = format!("{INDEXING_TASK_PREFIX}{}", indexing_task.pipeline_uid()); - let shards_str = shard_ids.iter().sorted().join(","); - let value = format!("{index_uid}:{source_id}:{shards_str}"); + let shard_ids_str = shard_ids.iter().sorted().join(","); + let value = format!("{index_uid}:{source_id}:{shard_ids_str}"); (key, value) } -fn parse_shards_str(shards_str: &str) -> Option> { - if shards_str.is_empty() { - return Some(Vec::new()); - } - let mut shard_ids = Vec::new(); - for shard_str in shards_str.split(',') { - let shard_id = shard_str.parse::().ok()?; - shard_ids.push(shard_id); - } - Some(shard_ids) +fn parse_shard_ids_str(shard_ids_str: &str) -> Vec { + shard_ids_str + .split(',') + .filter(|shard_id_str| !shard_id_str.is_empty()) + .map(ShardId::from) + .collect() } fn chitchat_kv_to_indexing_task(key: &str, value: &str) -> Option { @@ -495,7 +491,7 @@ fn chitchat_kv_to_indexing_task(key: &str, value: &str) -> Option let pipeline_uid = PipelineUid::from_str(pipeline_uid_str).ok()?; let (source_uid, shards_str) = value.rsplit_once(':')?; let (index_uid, source_id) = source_uid.rsplit_once(':')?; - let shard_ids = parse_shards_str(shards_str)?; + let shard_ids = parse_shard_ids_str(shards_str); Some(IndexingTask { index_uid: index_uid.to_string(), source_id: source_id.to_string(), @@ -1095,7 +1091,7 @@ mod tests { ); chitchat_guard.self_node_state().set( format!("{INDEXING_TASK_PREFIX}01BX5ZZKBKACTAV9WEVGEMMVS1"), - "my_index:uid:my_source:3a,5".to_string(), + "my_index-uid-my_source:3,5".to_string(), ); } node.wait_for_ready_members(|members| members.len() == 1, Duration::from_secs(5)) @@ -1211,7 +1207,7 @@ mod tests { pipeline_uid: Some(PipelineUid::from_u128(1u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![1, 2], + shard_ids: vec![ShardId::from(1), ShardId::from(2)], }], &mut node_state, ); @@ -1221,7 +1217,7 @@ mod tests { pipeline_uid: Some(PipelineUid::from_u128(2u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![1, 2, 3], + shard_ids: vec![ShardId::from(1), ShardId::from(2), ShardId::from(3)], }], &mut node_state, ); @@ -1231,13 +1227,13 @@ mod tests { pipeline_uid: Some(PipelineUid::from_u128(1u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![1, 2], + shard_ids: vec![ShardId::from(1), ShardId::from(2)], }, IndexingTask { pipeline_uid: Some(PipelineUid::from_u128(2u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![3, 4], + shard_ids: vec![ShardId::from(3), ShardId::from(4)], }, ], &mut node_state, @@ -1249,13 +1245,13 @@ mod tests { pipeline_uid: Some(PipelineUid::from_u128(1u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![1, 2], + shard_ids: vec![ShardId::from(1), ShardId::from(2)], }, IndexingTask { pipeline_uid: Some(PipelineUid::from_u128(2u128)), index_uid: "test:test2".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![3, 4], + shard_ids: vec![ShardId::from(3), ShardId::from(4)], }, ], &mut node_state, @@ -1267,13 +1263,13 @@ mod tests { pipeline_uid: Some(PipelineUid::from_u128(1u128)), index_uid: "test:test1".to_string(), source_id: "my-source1".to_string(), - shard_ids: vec![1, 2], + shard_ids: vec![ShardId::from(1), ShardId::from(2)], }, IndexingTask { pipeline_uid: Some(PipelineUid::from_u128(2u128)), index_uid: "test:test1".to_string(), source_id: "my-source2".to_string(), - shard_ids: vec![3, 4], + shard_ids: vec![ShardId::from(3), ShardId::from(4)], }, ], &mut node_state, @@ -1281,12 +1277,17 @@ mod tests { } #[test] - fn test_parse_shards_str() { - assert!(parse_shards_str("").unwrap().is_empty()); - assert_eq!(parse_shards_str("12").unwrap(), vec![12]); - assert_eq!(parse_shards_str("12,23").unwrap(), vec![12, 23]); - assert!(parse_shards_str("12,23,").is_none()); - assert!(parse_shards_str("12,23a,32").is_none()); + fn test_parse_shard_ids_str() { + assert!(parse_shard_ids_str("").is_empty()); + assert!(parse_shard_ids_str(",").is_empty()); + assert_eq!( + parse_shard_ids_str("00000000000000000012,"), + [ShardId::from(12)] + ); + assert_eq!( + parse_shard_ids_str("00000000000000000012,00000000000000000023,"), + [ShardId::from(12), ShardId::from(23)] + ); } #[test] @@ -1296,7 +1297,7 @@ mod tests { ); let task = super::chitchat_kv_to_indexing_task( "indexer.task:01BX5ZZKBKACTAV9WEVGEMMVS0", - "my_index:uid:my_source:1,3", + "my_index:uid:my_source:00000000000000000001,00000000000000000003", ) .unwrap(); assert_eq!( @@ -1305,6 +1306,6 @@ mod tests { ); assert_eq!(&task.index_uid, "my_index:uid"); assert_eq!(&task.source_id, "my_source"); - assert_eq!(&task.shard_ids, &[1, 3]); + assert_eq!(&task.shard_ids, &[ShardId::from(1), ShardId::from(3)]); } } diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 8510569e701..e15fb9ce0a5 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -228,7 +228,8 @@ impl Handler for ControlPlane { return Ok(()); }; let known_shard_ids: FnvHashSet = shard_entries - .map(|shard_entry| shard_entry.shard_id) + .map(|shard_entry| shard_entry.shard_id()) + .cloned() .collect(); // let's identify the shard that have reached EOF but have not yet been removed. let shard_ids_to_close: Vec = shard_positions_update @@ -286,7 +287,6 @@ fn convert_metastore_error( | MetastoreError::NotFound(_) => true, MetastoreError::Connection { .. } | MetastoreError::Db { .. } - | MetastoreError::InconsistentControlPlaneState { .. } | MetastoreError::Internal { .. } | MetastoreError::Io { .. } | MetastoreError::Unavailable(_) => false, @@ -942,11 +942,10 @@ mod tests { shards: vec![Shard { index_uid: "test-index:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, ..Default::default() }], - next_shard_id: 2, }]; let response = ListShardsResponse { subresponses }; Ok(response) @@ -982,7 +981,7 @@ mod tests { assert_eq!(subresponse.index_uid, "test-index:0"); assert_eq!(subresponse.source_id, INGEST_V2_SOURCE_ID); assert_eq!(subresponse.open_shards.len(), 1); - assert_eq!(subresponse.open_shards[0].shard_id, 1); + assert_eq!(subresponse.open_shards[0].shard_id(), ShardId::from(1)); universe.assert_quit().await; } @@ -1153,7 +1152,7 @@ mod tests { let subrequest = &delete_shards_request.subrequests[0]; assert_eq!(subrequest.index_uid, index_uid_clone); assert_eq!(subrequest.source_id, INGEST_V2_SOURCE_ID); - assert_eq!(&subrequest.shard_ids[..], &[17]); + assert_eq!(subrequest.shard_ids, [ShardId::from(17)]); Ok(DeleteShardsResponse {}) }, ); @@ -1161,7 +1160,7 @@ mod tests { let mut shard = Shard { index_uid: index_0.index_uid.to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 17, + shard_id: Some(ShardId::from(17)), leader_id: "test_node".to_string(), ..Default::default() }; @@ -1175,7 +1174,6 @@ mod tests { index_uid: index_uid_clone.to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), shards: vec![shard], - next_shard_id: 18, }], }; Ok(list_shards_resp) @@ -1200,7 +1198,7 @@ mod tests { control_plane_mailbox .ask(ShardPositionsUpdate { source_uid: source_uid.clone(), - shard_positions: vec![(17, Position::offset(1_000u64))], + shard_positions: vec![(ShardId::from(17), Position::offset(1_000u64))], }) .await .unwrap(); @@ -1216,14 +1214,14 @@ mod tests { .get("indexer-node-1") .unwrap(); assert_eq!(indexing_tasks.len(), 1); - assert_eq!(&indexing_tasks[0].shard_ids, &[17]); + assert_eq!(indexing_tasks[0].shard_ids, [ShardId::from(17)]); let _ = client_inbox.drain_for_test(); // This update should trigger the deletion of the shard and a new indexing plan. control_plane_mailbox .ask(ShardPositionsUpdate { source_uid, - shard_positions: vec![(17, Position::eof(1_000u64))], + shard_positions: vec![(ShardId::from(17), Position::eof(1_000u64))], }) .await .unwrap(); @@ -1288,7 +1286,7 @@ mod tests { let subrequest = &delete_shards_request.subrequests[0]; assert_eq!(subrequest.index_uid, index_uid_clone); assert_eq!(subrequest.source_id, INGEST_V2_SOURCE_ID); - assert_eq!(&subrequest.shard_ids[..], &[17]); + assert_eq!(subrequest.shard_ids, [ShardId::from(17)]); Ok(DeleteShardsResponse {}) }, ); @@ -1301,7 +1299,6 @@ mod tests { index_uid: index_uid_clone.to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), shards: vec![], - next_shard_id: 18, }], }; Ok(list_shards_resp) @@ -1326,7 +1323,7 @@ mod tests { control_plane_mailbox .ask(ShardPositionsUpdate { source_uid: source_uid.clone(), - shard_positions: vec![(17, Position::eof(1_000u64))], + shard_positions: vec![(ShardId::from(17), Position::eof(1_000u64))], }) .await .unwrap(); @@ -1377,14 +1374,13 @@ mod tests { shards: vec![Shard { index_uid: index_uid_clone.to_string(), source_id: source.source_id.to_string(), - shard_id: 15, + shard_id: Some(ShardId::from(15)), leader_id: "node1".to_string(), follower_id: None, shard_state: ShardState::Open as i32, publish_position_inclusive: None, publish_token: None, }], - next_shard_id: 18, }], }; Ok(list_shards_resp) @@ -1394,10 +1390,12 @@ mod tests { .expect_retain_shards() .times(1) .in_sequence(&mut seq) - .returning(|mut request| { + .returning(|request| { assert_eq!(request.retain_shards_for_sources.len(), 1); - let retain_shards_for_source = request.retain_shards_for_sources.pop().unwrap(); - assert_eq!(&retain_shards_for_source.shard_ids, &[15]); + assert_eq!( + request.retain_shards_for_sources[0].shard_ids, + [ShardId::from(15)] + ); Ok(RetainShardsResponse {}) }); @@ -1454,10 +1452,12 @@ mod tests { ingester_mock .expect_retain_shards() .times(2) - .returning(|mut request| { + .returning(|request| { assert_eq!(request.retain_shards_for_sources.len(), 1); - let retain_shards_for_source = request.retain_shards_for_sources.pop().unwrap(); - assert_eq!(&retain_shards_for_source.shard_ids, &[15]); + assert_eq!( + request.retain_shards_for_sources[0].shard_ids, + [ShardId::from(15)] + ); Ok(RetainShardsResponse {}) }); ingester_pool.insert("node1".into(), ingester_mock.into()); @@ -1499,14 +1499,13 @@ mod tests { shards: vec![Shard { index_uid: index_uid_clone.to_string(), source_id: source.source_id.to_string(), - shard_id: 15, + shard_id: Some(ShardId::from(15)), leader_id: "node1".to_string(), follower_id: None, shard_state: ShardState::Open as i32, publish_position_inclusive: None, publish_token: None, }], - next_shard_id: 18, }], }; Ok(list_shards_resp) diff --git a/quickwit/quickwit-control-plane/src/indexing_plan.rs b/quickwit/quickwit-control-plane/src/indexing_plan.rs index 6932c1e32a1..7f7b6f309a5 100644 --- a/quickwit/quickwit-control-plane/src/indexing_plan.rs +++ b/quickwit/quickwit-control-plane/src/indexing_plan.rs @@ -69,21 +69,13 @@ impl PhysicalIndexingPlan { for task in tasks.iter_mut() { task.shard_ids.sort_unstable(); } - tasks.sort_by(|left, right| { + tasks.sort_unstable_by(|left, right| { left.index_uid .cmp(&right.index_uid) .then_with(|| left.source_id.cmp(&right.source_id)) - .then_with(|| { - left.shard_ids - .first() - .copied() - .cmp(&right.shard_ids.first().copied()) - }) - .then_with(|| left.pipeline_uid().cmp(&right.pipeline_uid())) + .then_with(|| left.shard_ids.first().cmp(&right.shard_ids.first())) + .then_with(|| left.pipeline_uid.cmp(&right.pipeline_uid)) }); - for task in tasks { - task.shard_ids.sort(); - } } } } diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs index 11fcf287596..57f77e655c5 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs @@ -144,7 +144,8 @@ fn get_sources_to_schedule(model: &ControlPlaneModel) -> Vec { let shard_ids: Vec = model .list_shards_for_source(&source_uid) .expect("source should exist") - .map(|shard| shard.shard_id) + .map(|shard| shard.shard_id()) + .cloned() .collect(); if shard_ids.is_empty() { continue; @@ -749,11 +750,11 @@ mod tests { let shard = Shard { index_uid: index_uid.to_string(), source_id: "ingest_v2".to_string(), - shard_id: 17, + shard_id: Some(ShardId::from(17)), shard_state: ShardState::Open as i32, ..Default::default() }; - model.insert_newly_opened_shards(&index_uid, &"ingest_v2".to_string(), vec![shard], 18); + model.insert_newly_opened_shards(&index_uid, &"ingest_v2".to_string(), vec![shard]); let shards: Vec = get_sources_to_schedule(&model); assert_eq!(shards.len(), 3); } diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/scheduling/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/scheduling/mod.rs index 9ae4c7a4183..4189cdf90b9 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/scheduling/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/scheduling/mod.rs @@ -232,9 +232,9 @@ fn convert_scheduling_solution_to_physical_plan_single_node_single_source( let shard_ids: Vec = previous_task .shard_ids .iter() - .copied() .filter(|shard_id| shard_ids.contains(shard_id)) .take(max_shard_in_pipeline) + .cloned() .collect(); remaining_num_shards_to_schedule_on_node -= shard_ids.len() as u32; let new_task = IndexingTask { @@ -384,8 +384,8 @@ fn convert_scheduling_solution_to_physical_plan( if indexing_task.index_uid == source.source_uid.index_uid.as_str() && indexing_task.source_id == source.source_uid.source_id { - indexing_task.shard_ids.retain(|&shard| { - let shard_added = scheduled_shards.insert(shard); + indexing_task.shard_ids.retain(|shard_id| { + let shard_added = scheduled_shards.insert(shard_id.clone()); if shard_added { true } else { @@ -405,7 +405,7 @@ fn convert_scheduling_solution_to_physical_plan( let missing_shards = shard_ids .iter() .filter(|shard_id| !scheduled_shards.contains(shard_id)) - .copied(); + .cloned(); // Let's assign the missing shards. @@ -621,7 +621,16 @@ mod tests { let source_0 = SourceToSchedule { source_uid: source_uid0.clone(), source_type: SourceToScheduleType::Sharded { - shard_ids: vec![0, 1, 2, 3, 4, 5, 6, 7], + shard_ids: vec![ + ShardId::from(0), + ShardId::from(1), + ShardId::from(2), + ShardId::from(3), + ShardId::from(4), + ShardId::from(5), + ShardId::from(6), + ShardId::from(7), + ], load_per_shard: NonZeroU32::new(1_000).unwrap(), }, }; @@ -660,11 +669,20 @@ mod tests { assert_eq!(node2_plan.len(), 4); assert_eq!(&node2_plan[0].source_id, &source_uid0.source_id); - assert_eq!(&node2_plan[0].shard_ids, &[0, 1, 2]); + assert_eq!( + &node2_plan[0].shard_ids, + &[ShardId::from(0), ShardId::from(1), ShardId::from(2)] + ); assert_eq!(&node2_plan[1].source_id, &source_uid0.source_id); - assert_eq!(&node2_plan[1].shard_ids, &[3, 4, 5]); + assert_eq!( + &node2_plan[1].shard_ids, + &[ShardId::from(3), ShardId::from(4), ShardId::from(5)] + ); assert_eq!(&node2_plan[2].source_id, &source_uid0.source_id); - assert_eq!(&node2_plan[2].shard_ids, &[6, 7]); + assert_eq!( + &node2_plan[2].shard_ids, + &[ShardId::from(6), ShardId::from(7)] + ); assert_eq!(&node2_plan[3].source_id, &source_uid2.source_id); } @@ -727,14 +745,26 @@ mod tests { let indexing_tasks = make_indexing_tasks( &source_uid, &[ - (PipelineUid::from_u128(1u128), &[1, 2]), - (PipelineUid::from_u128(2u128), &[3, 4, 5]), + ( + PipelineUid::from_u128(1u128), + &[ShardId::from(1), ShardId::from(2)], + ), + ( + PipelineUid::from_u128(2u128), + &[ShardId::from(3), ShardId::from(4), ShardId::from(5)], + ), ], ); let sources = vec![SourceToSchedule { source_uid: source_uid.clone(), source_type: SourceToScheduleType::Sharded { - shard_ids: vec![0, 1, 3, 4, 5], + shard_ids: vec![ + ShardId::from(0), + ShardId::from(1), + ShardId::from(3), + ShardId::from(4), + ShardId::from(5), + ], load_per_shard: NonZeroU32::new(1_000).unwrap(), }, }]; @@ -751,13 +781,19 @@ mod tests { ); let indexing_tasks = new_plan.indexer("node1").unwrap(); assert_eq!(indexing_tasks.len(), 2); - assert_eq!(&indexing_tasks[0].shard_ids, &[0, 1]); - assert_eq!(&indexing_tasks[1].shard_ids, &[3, 4, 5]); + assert_eq!( + &indexing_tasks[0].shard_ids, + &[ShardId::from(0), ShardId::from(1)] + ); + assert_eq!( + &indexing_tasks[1].shard_ids, + &[ShardId::from(3), ShardId::from(4), ShardId::from(5)] + ); } fn group_shards_into_pipelines_aux( source_uid: &SourceUid, - shard_ids: &[ShardId], + shard_ids: &[u64], previous_pipeline_shards: &[(PipelineUid, &[ShardId])], load_per_shard: CpuCapacity, ) -> Vec { @@ -765,7 +801,7 @@ mod tests { let sources = vec![SourceToSchedule { source_uid: source_uid.clone(), source_type: SourceToScheduleType::Sharded { - shard_ids: shard_ids.to_vec(), + shard_ids: shard_ids.iter().copied().map(ShardId::from).collect(), load_per_shard: NonZeroU32::new(load_per_shard.cpu_millis()).unwrap(), }, }]; @@ -783,7 +819,7 @@ mod tests { ); let mut indexing_tasks = new_plan.indexer(NODE).unwrap().to_vec(); // We sort indexing tasks for normalization purpose - indexing_tasks.sort_by_key(|task| task.shard_ids[0]); + indexing_tasks.sort_by_key(|task| task.shard_ids[0].clone()); indexing_tasks } @@ -805,8 +841,23 @@ mod tests { mcpu(400), ); assert_eq!(indexing_tasks_1.len(), 2); - assert_eq!(&indexing_tasks_1[0].shard_ids, &[0, 1, 2, 3, 4, 5, 6, 7]); - assert_eq!(&indexing_tasks_1[1].shard_ids, &[8, 9, 10]); + assert_eq!( + &indexing_tasks_1[0].shard_ids, + &[ + ShardId::from(0), + ShardId::from(1), + ShardId::from(2), + ShardId::from(3), + ShardId::from(4), + ShardId::from(5), + ShardId::from(6), + ShardId::from(7) + ] + ); + assert_eq!( + &indexing_tasks_1[1].shard_ids, + &[ShardId::from(8), ShardId::from(9), ShardId::from(10)] + ); let pipeline_tasks1: Vec<(PipelineUid, &[ShardId])> = indexing_tasks_1 .iter() @@ -821,9 +872,27 @@ mod tests { mcpu(600), ); assert_eq!(indexing_tasks_2.len(), 3); - assert_eq!(&indexing_tasks_2[0].shard_ids, &[0, 1, 2, 3, 4]); - assert_eq!(&indexing_tasks_2[1].shard_ids, &[5, 6, 8, 9, 10]); - assert_eq!(&indexing_tasks_2[2].shard_ids, &[7]); + assert_eq!( + &indexing_tasks_2[0].shard_ids, + &[ + ShardId::from(0), + ShardId::from(1), + ShardId::from(2), + ShardId::from(3), + ShardId::from(4) + ] + ); + assert_eq!( + &indexing_tasks_2[1].shard_ids, + &[ + ShardId::from(5), + ShardId::from(6), + ShardId::from(8), + ShardId::from(9), + ShardId::from(10) + ] + ); + assert_eq!(&indexing_tasks_2[2].shard_ids, &[ShardId::from(7)]); // Now the load comes back to normal // The hysteresis takes effect. We do not switch back to 2 pipelines. @@ -852,8 +921,27 @@ mod tests { mcpu(200), ); assert_eq!(indexing_tasks_4.len(), 2); - assert_eq!(&indexing_tasks_4[0].shard_ids, &[0, 1, 2, 3, 4, 7]); - assert_eq!(&indexing_tasks_4[1].shard_ids, &[5, 6, 8, 9, 10]); + assert_eq!( + &indexing_tasks_4[0].shard_ids, + &[ + ShardId::from(0), + ShardId::from(1), + ShardId::from(2), + ShardId::from(3), + ShardId::from(4), + ShardId::from(7) + ] + ); + assert_eq!( + &indexing_tasks_4[1].shard_ids, + &[ + ShardId::from(5), + ShardId::from(6), + ShardId::from(8), + ShardId::from(9), + ShardId::from(10) + ] + ); } /// We want to make sure for small pipelines, we still reschedule them with the same @@ -865,12 +953,12 @@ mod tests { let indexing_tasks = group_shards_into_pipelines_aux( &source_uid, &[12], - &[(pipeline_uid, &[12])], + &[(pipeline_uid, &[ShardId::from(12)])], mcpu(100), ); assert_eq!(indexing_tasks.len(), 1); let indexing_task = &indexing_tasks[0]; - assert_eq!(&indexing_task.shard_ids, &[12]); + assert_eq!(&indexing_task.shard_ids, &[ShardId::from(12)]); assert_eq!(indexing_task.pipeline_uid.unwrap(), pipeline_uid); } @@ -908,7 +996,7 @@ mod tests { source_id: "_ingest-source".to_string(), }, source_type: SourceToScheduleType::Sharded { - shard_ids: vec![1], + shard_ids: vec![ShardId::from(1)], load_per_shard: NonZeroU32::new(250).unwrap(), }, }, @@ -928,19 +1016,30 @@ mod tests { index_uid: source_uid.index_uid.to_string(), source_id: source_uid.source_id.to_string(), pipeline_uid: Some(PipelineUid::new()), - shard_ids: vec![1, 4, 5], + shard_ids: vec![ShardId::from(1), ShardId::from(4), ShardId::from(5)], }; let previous_task2 = IndexingTask { index_uid: source_uid.index_uid.to_string(), source_id: source_uid.source_id.to_string(), pipeline_uid: Some(PipelineUid::new()), - shard_ids: vec![6, 7, 8, 9, 10], + shard_ids: vec![ + ShardId::from(6), + ShardId::from(7), + ShardId::from(8), + ShardId::from(9), + ShardId::from(10), + ], }; { let sharded_source = SourceToSchedule { source_uid: source_uid.clone(), source_type: SourceToScheduleType::Sharded { - shard_ids: vec![1, 2, 4, 6], + shard_ids: vec![ + ShardId::from(1), + ShardId::from(2), + ShardId::from(4), + ShardId::from(6), + ], load_per_shard: NonZeroU32::new(1_000).unwrap(), }, }; @@ -950,17 +1049,22 @@ mod tests { &sharded_source, ); assert_eq!(tasks.len(), 2); - assert_eq!(&tasks[0].index_uid, source_uid.index_uid.as_str()); - assert_eq!(&tasks[0].shard_ids, &[1, 4]); - assert_eq!(&tasks[1].index_uid, source_uid.index_uid.as_str()); - assert_eq!(&tasks[1].shard_ids, &[6]); + assert_eq!(tasks[0].index_uid, source_uid.index_uid.as_str()); + assert_eq!(tasks[0].shard_ids, [ShardId::from(1), ShardId::from(4)]); + assert_eq!(tasks[1].index_uid, source_uid.index_uid.as_str()); + assert_eq!(tasks[1].shard_ids, [ShardId::from(6)]); } { // smaller shards force a merge into a single pipeline let sharded_source = SourceToSchedule { source_uid: source_uid.clone(), source_type: SourceToScheduleType::Sharded { - shard_ids: vec![1, 2, 4, 6], + shard_ids: vec![ + ShardId::from(1), + ShardId::from(2), + ShardId::from(4), + ShardId::from(6), + ], load_per_shard: NonZeroU32::new(250).unwrap(), }, }; @@ -970,8 +1074,8 @@ mod tests { &sharded_source, ); assert_eq!(tasks.len(), 1); - assert_eq!(&tasks[0].index_uid, source_uid.index_uid.as_str()); - assert_eq!(&tasks[0].shard_ids, &[1, 4]); + assert_eq!(tasks[0].index_uid, source_uid.index_uid.as_str()); + assert_eq!(tasks[0].shard_ids, [ShardId::from(1), ShardId::from(4)]); } } @@ -986,14 +1090,14 @@ mod tests { index_uid: source_uid.index_uid.to_string(), source_id: source_uid.source_id.to_string(), pipeline_uid: Some(pipeline_uid1), - shard_ids: vec![], + shard_ids: Vec::new(), }; let pipeline_uid2 = PipelineUid::new(); let previous_task2 = IndexingTask { index_uid: source_uid.index_uid.to_string(), source_id: source_uid.source_id.to_string(), pipeline_uid: Some(pipeline_uid2), - shard_ids: vec![], + shard_ids: Vec::new(), }; { let sharded_source = SourceToSchedule { @@ -1009,8 +1113,8 @@ mod tests { &sharded_source, ); assert_eq!(tasks.len(), 1); - assert_eq!(&tasks[0].index_uid, source_uid.index_uid.as_str()); - assert!(&tasks[0].shard_ids.is_empty()); + assert_eq!(tasks[0].index_uid, source_uid.index_uid.as_str()); + assert!(tasks[0].shard_ids.is_empty()); assert_eq!(tasks[0].pipeline_uid.as_ref().unwrap(), &pipeline_uid1); } { @@ -1042,11 +1146,11 @@ mod tests { &sharded_source, ); assert_eq!(tasks.len(), 2); - assert_eq!(&tasks[0].index_uid, source_uid.index_uid.as_str()); - assert!(&tasks[0].shard_ids.is_empty()); + assert_eq!(tasks[0].index_uid, source_uid.index_uid.as_str()); + assert!(tasks[0].shard_ids.is_empty()); assert_eq!(tasks[0].pipeline_uid.as_ref().unwrap(), &pipeline_uid1); - assert_eq!(&tasks[1].index_uid, source_uid.index_uid.as_str()); - assert!(&tasks[1].shard_ids.is_empty()); + assert_eq!(tasks[1].index_uid, source_uid.index_uid.as_str()); + assert!(tasks[1].shard_ids.is_empty()); assert_eq!(tasks[1].pipeline_uid.as_ref().unwrap(), &pipeline_uid2); } { @@ -1063,11 +1167,11 @@ mod tests { &sharded_source, ); assert_eq!(tasks.len(), 2); - assert_eq!(&tasks[0].index_uid, source_uid.index_uid.as_str()); - assert!(&tasks[0].shard_ids.is_empty()); + assert_eq!(tasks[0].index_uid, source_uid.index_uid.as_str()); + assert!(tasks[0].shard_ids.is_empty()); assert_eq!(tasks[0].pipeline_uid.as_ref().unwrap(), &pipeline_uid1); - assert_eq!(&tasks[1].index_uid, source_uid.index_uid.as_str()); - assert!(&tasks[1].shard_ids.is_empty()); + assert_eq!(tasks[1].index_uid, source_uid.index_uid.as_str()); + assert!(tasks[1].shard_ids.is_empty()); assert_ne!(tasks[1].pipeline_uid.as_ref().unwrap(), &pipeline_uid1); } } diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index ebd6e9fe3bb..76d47e67f9e 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -42,6 +42,7 @@ use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceUid}; use rand::seq::SliceRandom; use tokio::time::timeout; use tracing::{error, info, warn}; +use ulid::Ulid; use crate::ingest::wait_handle::WaitHandle; use crate::metrics::CONTROL_PLANE_METRICS; @@ -147,7 +148,7 @@ impl IngestController { let shards_for_source = RetainShardsForSource { index_uid: source_uid.index_uid.to_string(), source_id: source_uid.source_id.clone(), - shard_ids: shard_ids.iter().copied().collect(), + shard_ids: shard_ids.iter().cloned().collect(), }; retain_shards_req .retain_shards_for_sources @@ -378,7 +379,7 @@ impl IngestController { get_or_create_open_shards_failures.push(get_or_create_open_shards_failure); continue; }; - let Some((open_shard_entries, next_shard_id)) = model.find_open_shards( + let Some(open_shard_entries) = model.find_open_shards( &index_uid, &get_open_shards_subrequest.source_id, &unavailable_leaders, @@ -413,13 +414,14 @@ impl IngestController { .ok_or_else(|| { ControlPlaneError::Unavailable("no ingester available".to_string()) })?; + let shard_id = ShardId::from(Ulid::new()); let open_shards_subrequest = metastore::OpenShardsSubrequest { subrequest_id: get_open_shards_subrequest.subrequest_id, index_uid: index_uid.into(), source_id: get_open_shards_subrequest.source_id, + shard_id: Some(shard_id), leader_id: leader_id.into(), follower_id: follower_id.map(|follower_id| follower_id.into()), - next_shard_id, }; open_shards_subrequests.push(open_shards_subrequest); } @@ -442,9 +444,8 @@ impl IngestController { &index_uid, &source_id, open_shards_subresponse.opened_shards, - open_shards_subresponse.next_shard_id, ); - if let Some((open_shard_entries, _next_shard_id)) = + if let Some(open_shard_entries) = model.find_open_shards(&index_uid, &source_id, &unavailable_leaders) { let open_shards = open_shard_entries @@ -524,11 +525,6 @@ impl IngestController { source_id=%source_uid.source_id, "scaling up number of shards to {new_num_open_shards}" ); - // Expect: the source should exist because we just acquired a permit. - let next_shard_id = model - .next_shard_id(&source_uid) - .expect("source should exist"); - let mut unavailable_leaders: FnvHashSet = FnvHashSet::default(); let Some((leader_id, follower_id)) = self @@ -539,13 +535,14 @@ impl IngestController { model.release_scaling_permits(&source_uid, ScalingMode::Up, NUM_PERMITS); return; }; + let shard_id = ShardId::from(Ulid::new()); let open_shards_subrequest = metastore::OpenShardsSubrequest { subrequest_id: 0, index_uid: source_uid.index_uid.clone().into(), source_id: source_uid.source_id.clone(), + shard_id: Some(shard_id), leader_id: leader_id.into(), follower_id: follower_id.map(Into::into), - next_shard_id, }; let open_shards_request = metastore::OpenShardsRequest { subrequests: vec![open_shards_subrequest], @@ -574,7 +571,6 @@ impl IngestController { &index_uid, &source_id, open_shards_subresponse.opened_shards, - open_shards_subresponse.next_shard_id, ); } let label_values = [source_uid.index_uid.index_id(), &source_uid.source_id]; @@ -619,7 +615,7 @@ impl IngestController { let shards = vec![ShardIds { index_uid: source_uid.index_uid.clone().into(), source_id: source_uid.source_id.clone(), - shard_ids: vec![shard_id], + shard_ids: vec![shard_id.clone()], }]; let close_shards_request = CloseShardsRequest { shards }; @@ -642,7 +638,8 @@ impl IngestController { } /// Finds the shard with the highest ingestion rate on the ingester with the least number of open -/// shards. +/// shards. If multiple shards have the same ingestion rate, the shard with the highest shard ID is +/// chosen. fn find_scale_down_candidate( source_uid: &SourceUid, model: &ControlPlaneModel, @@ -656,7 +653,12 @@ fn find_scale_down_candidate( .and_modify(|(num_shards, candidate)| { *num_shards += 1; - if candidate.ingestion_rate < shard.ingestion_rate { + if shard + .ingestion_rate + .cmp(&candidate.ingestion_rate) + .then_with(|| shard.shard_id.cmp(&candidate.shard_id)) + .is_gt() + { *candidate = shard; } }) @@ -666,7 +668,9 @@ fn find_scale_down_candidate( per_leader_candidates .into_iter() .min_by_key(|(_leader_id, (num_shards, _shard))| *num_shards) - .map(|(leader_id, (_num_shards, shard))| (leader_id.clone().into(), shard.shard_id)) + .map(|(leader_id, (_num_shards, shard))| { + (leader_id.clone().into(), shard.shard_id().clone()) + }) } #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -927,12 +931,11 @@ mod tests { opened_shards: vec![Shard { index_uid: index_uid_1.clone().into(), source_id: source_id.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-2".to_string(), ..Default::default() }], - next_shard_id: 2, }]; let response = metastore::OpenShardsResponse { subresponses }; Ok(response) @@ -958,7 +961,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index-1:0"); assert_eq!(request.shards[0].source_id, "test-source"); - assert_eq!(request.shards[0].shard_id, 1); + assert_eq!(request.shards[0].shard_id(), ShardId::from(1)); assert_eq!(request.shards[0].leader_id, "test-ingester-2"); Ok(InitShardsResponse {}) @@ -988,7 +991,7 @@ mod tests { Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -996,14 +999,14 @@ mod tests { Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), leader_id: "test-ingester-1".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }, ]; - model.insert_newly_opened_shards(&index_uid_0, &source_id.into(), shards, 3); + model.insert_newly_opened_shards(&index_uid_0, &source_id.into(), shards); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), @@ -1061,7 +1064,7 @@ mod tests { assert_eq!(success.index_uid, index_uid_0.as_str()); assert_eq!(success.source_id, source_id); assert_eq!(success.open_shards.len(), 1); - assert_eq!(success.open_shards[0].shard_id, 2); + assert_eq!(success.open_shards[0].shard_id(), ShardId::from(2)); assert_eq!(success.open_shards[0].leader_id, "test-ingester-1"); let success = &response.successes[1]; @@ -1069,7 +1072,7 @@ mod tests { assert_eq!(success.index_uid, index_uid_1.as_str()); assert_eq!(success.source_id, source_id); assert_eq!(success.open_shards.len(), 1); - assert_eq!(success.open_shards[0].shard_id, 1); + assert_eq!(success.open_shards[0].shard_id(), ShardId::from(1)); assert_eq!(success.open_shards[0].leader_id, "test-ingester-2"); let failure = &response.failures[0]; @@ -1107,21 +1110,21 @@ mod tests { let source_id: SourceId = "test-source".into(); let shards = vec![Shard { - shard_id: 1, + shard_id: Some(ShardId::from(1)), index_uid: index_uid.to_string(), source_id: source_id.clone(), leader_id: "test-ingester-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 3); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), closed_shards: vec![ShardIds { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_ids: vec![1, 2], + shard_ids: vec![ShardId::from(1), ShardId::from(2)], }], unavailable_leaders: Vec::new(), }; @@ -1134,7 +1137,7 @@ mod tests { let shard_1 = model .all_shards() - .find(|shard| shard.shard_id == 1) + .find(|shard| shard.shard_id() == ShardId::from(1)) .unwrap(); assert!(shard_1.is_closed()); } @@ -1158,31 +1161,31 @@ mod tests { let shards = vec![ Shard { - shard_id: 1, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }, Shard { - shard_id: 2, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(2)), leader_id: "test-ingester-0".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() }, Shard { - shard_id: 3, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(3)), leader_id: "test-ingester-1".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }, ]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 4); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let request = GetOrCreateOpenShardsRequest { subrequests: Vec::new(), @@ -1198,19 +1201,19 @@ mod tests { let shard_1 = model .all_shards() - .find(|shard| shard.shard_id == 1) + .find(|shard| shard.shard_id() == ShardId::from(1)) .unwrap(); assert!(shard_1.is_unavailable()); let shard_2 = model .all_shards() - .find(|shard| shard.shard_id == 2) + .find(|shard| shard.shard_id() == ShardId::from(2)) .unwrap(); assert!(shard_2.is_closed()); let shard_3 = model .all_shards() - .find(|shard| shard.shard_id == 3) + .find(|shard| shard.shard_id() == ShardId::from(3)) .unwrap(); assert!(shard_3.is_open()); } @@ -1237,12 +1240,12 @@ mod tests { let shards = vec![Shard { index_uid: index_uid.to_string(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 2); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let shard_entries: Vec = model.all_shards().cloned().collect(); assert_eq!(shard_entries.len(), 1); @@ -1250,7 +1253,7 @@ mod tests { // Test update shard ingestion rate but no scale down because num open shards is 1. let shard_infos = BTreeSet::from_iter([ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(1), }]); @@ -1269,14 +1272,14 @@ mod tests { // Test update shard ingestion rate with failing scale down. let shards = vec![Shard { - shard_id: 2, index_uid: index_uid.to_string(), source_id: source_id.clone(), - leader_id: "test-ingester".to_string(), + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester".to_string(), ..Default::default() }]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 2); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let shard_entries: Vec = model.all_shards().cloned().collect(); assert_eq!(shard_entries.len(), 2); @@ -1287,7 +1290,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index:0"); assert_eq!(request.shards[0].source_id, "test-source"); - assert_eq!(request.shards[0].shard_ids, vec![1]); + assert_eq!(request.shards[0].shard_ids, [ShardId::from(2)]); Err(IngestV2Error::Internal( "failed to close shards".to_string(), @@ -1302,12 +1305,12 @@ mod tests { let shard_infos = BTreeSet::from_iter([ ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(1), }, ShardInfo { - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(1), }, @@ -1324,12 +1327,12 @@ mod tests { // Test update shard ingestion rate with failing scale up. let shard_infos = BTreeSet::from_iter([ ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(4), }, ShardInfo { - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(4), }, @@ -1356,7 +1359,6 @@ mod tests { assert_eq!(request.subrequests[0].index_uid, "test-index:0"); assert_eq!(request.subrequests[0].source_id, INGEST_V2_SOURCE_ID); assert_eq!(request.subrequests[0].leader_id, "test-ingester"); - assert_eq!(request.subrequests[0].next_shard_id, 1); Err(MetastoreError::InvalidArgument { message: "failed to open shards".to_string(), @@ -1367,7 +1369,6 @@ mod tests { assert_eq!(request.subrequests[0].index_uid, "test-index:0"); assert_eq!(request.subrequests[0].source_id, INGEST_V2_SOURCE_ID); assert_eq!(request.subrequests[0].leader_id, "test-ingester"); - assert_eq!(request.subrequests[0].next_shard_id, 1); let subresponses = vec![metastore::OpenShardsSubresponse { subrequest_id: 0, @@ -1376,12 +1377,11 @@ mod tests { opened_shards: vec![Shard { index_uid: "test-index:0".into(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }], - next_shard_id: 2, }]; let response = metastore::OpenShardsResponse { subresponses }; Ok(response) @@ -1436,7 +1436,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index:0"); assert_eq!(request.shards[0].source_id, INGEST_V2_SOURCE_ID); - assert_eq!(request.shards[0].shard_id, 1); + assert_eq!(request.shards[0].shard_id(), ShardId::from(1)); assert_eq!(request.shards[0].leader_id, "test-ingester"); Err(IngestV2Error::Internal("failed to init shards".to_string())) @@ -1445,7 +1445,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index:0"); assert_eq!(request.shards[0].source_id, INGEST_V2_SOURCE_ID); - assert_eq!(request.shards[0].shard_id, 1); + assert_eq!(request.shards[0].shard_id(), ShardId::from(1)); assert_eq!(request.shards[0].leader_id, "test-ingester"); Ok(InitShardsResponse {}) @@ -1503,14 +1503,14 @@ mod tests { .await; let shards = vec![Shard { - shard_id: 1, + shard_id: Some(ShardId::from(1)), index_uid: index_uid.to_string(), source_id: source_id.clone(), leader_id: "test-ingester".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 2); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); // Test ingester is unavailable. ingest_controller @@ -1526,7 +1526,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index:0"); assert_eq!(request.shards[0].source_id, "test-source"); - assert_eq!(request.shards[0].shard_ids, vec![1]); + assert_eq!(request.shards[0].shard_ids, [ShardId::from(1)]); Err(IngestV2Error::Internal( "failed to close shards".to_string(), @@ -1539,7 +1539,7 @@ mod tests { assert_eq!(request.shards.len(), 1); assert_eq!(request.shards[0].index_uid, "test-index:0"); assert_eq!(request.shards[0].source_id, "test-source"); - assert_eq!(request.shards[0].shard_ids, vec![1]); + assert_eq!(request.shards[0].shard_ids, [ShardId::from(1)]); Ok(CloseShardsResponse {}) }); @@ -1558,14 +1558,14 @@ mod tests { assert!(model.all_shards().all(|shard| shard.is_closed())); let shards = vec![Shard { - shard_id: 2, + shard_id: Some(ShardId::from(2)), index_uid: index_uid.to_string(), source_id: source_id.clone(), leader_id: "test-ingester".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 3); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); // Test rate limited. ingest_controller @@ -1589,84 +1589,84 @@ mod tests { let shards = vec![ Shard { - shard_id: 1, - leader_id: "test-ingester-0".to_string(), index_uid: index_uid.clone().into(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), ..Default::default() }, Shard { - shard_id: 2, index_uid: index_uid.clone().into(), source_id: source_id.clone(), - leader_id: "test-ingester-0".to_string(), + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-0".to_string(), ..Default::default() }, Shard { - shard_id: 3, index_uid: index_uid.clone().into(), source_id: source_id.clone(), - leader_id: "test-ingester-0".to_string(), + shard_id: Some(ShardId::from(3)), shard_state: ShardState::Closed as i32, + leader_id: "test-ingester-0".to_string(), ..Default::default() }, Shard { - shard_id: 4, index_uid: index_uid.clone().into(), source_id: source_id.clone(), - leader_id: "test-ingester-1".to_string(), + shard_id: Some(ShardId::from(4)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), ..Default::default() }, Shard { - shard_id: 5, index_uid: index_uid.clone().into(), source_id: source_id.clone(), - leader_id: "test-ingester-1".to_string(), + shard_id: Some(ShardId::from(5)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), ..Default::default() }, Shard { - shard_id: 6, index_uid: index_uid.clone().into(), source_id: source_id.clone(), - leader_id: "test-ingester-1".to_string(), + shard_id: Some(ShardId::from(6)), shard_state: ShardState::Open as i32, + leader_id: "test-ingester-1".to_string(), ..Default::default() }, ]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 7); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let shard_infos = BTreeSet::from_iter([ ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(1), }, ShardInfo { - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(2), }, ShardInfo { - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(3), }, ShardInfo { - shard_id: 4, + shard_id: ShardId::from(4), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(4), }, ShardInfo { - shard_id: 5, + shard_id: ShardId::from(5), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(5), }, ShardInfo { - shard_id: 6, + shard_id: ShardId::from(6), shard_state: ShardState::Open, ingestion_rate: quickwit_ingest::RateMibPerSec(6), }, @@ -1675,7 +1675,7 @@ mod tests { let (leader_id, shard_id) = find_scale_down_candidate(&source_uid, &model).unwrap(); assert_eq!(leader_id, "test-ingester-0"); - assert_eq!(shard_id, 2); + assert_eq!(shard_id, ShardId::from(2)); } #[tokio::test] @@ -1692,34 +1692,34 @@ mod tests { let mut model = ControlPlaneModel::default(); let shards = vec![ Shard { - shard_id: 1, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, leader_id: "node-1".to_string(), follower_id: Some("node-2".to_string()), - shard_state: ShardState::Open as i32, ..Default::default() }, Shard { - shard_id: 2, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(2)), + shard_state: ShardState::Open as i32, leader_id: "node-2".to_string(), follower_id: Some("node-3".to_string()), - shard_state: ShardState::Open as i32, ..Default::default() }, Shard { - shard_id: 3, index_uid: index_uid.to_string(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(3)), + shard_state: ShardState::Open as i32, leader_id: "node-2".to_string(), follower_id: Some("node-1".to_string()), - shard_state: ShardState::Open as i32, ..Default::default() }, ]; - model.insert_newly_opened_shards(&index_uid, &source_id, shards, 2); + model.insert_newly_opened_shards(&index_uid, &source_id, shards); let mut ingester_mock1 = IngesterServiceClient::mock(); let ingester_mock2 = IngesterServiceClient::mock(); @@ -1730,11 +1730,13 @@ mod tests { ingester_mock1 .expect_retain_shards() .once() - .returning(move |mut request| { + .returning(move |request| { assert_eq!(request.retain_shards_for_sources.len(), 1); - let retain_shards_for_source = request.retain_shards_for_sources.pop().unwrap(); - assert_eq!(&retain_shards_for_source.shard_ids, &[1, 3]); - count_calls_clone.fetch_add(1, Ordering::SeqCst); + assert_eq!( + request.retain_shards_for_sources[0].shard_ids, + [ShardId::from(1), ShardId::from(3)] + ); + count_calls_clone.fetch_add(1, Ordering::Release); Ok(RetainShardsResponse {}) }); ingester_pool.insert("node-1".into(), ingester_mock1.into()); @@ -1743,6 +1745,6 @@ mod tests { let node_id = "node-1".into(); let wait_handle = ingest_controller.sync_with_ingester(&node_id, &model); wait_handle.wait().await; - assert_eq!(count_calls.load(Ordering::SeqCst), 1); + assert_eq!(count_calls.load(Ordering::Acquire), 1); } } diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 6c07b7adb88..7007ef4bdc4 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -38,7 +38,7 @@ use quickwit_proto::metastore::{ }; use quickwit_proto::types::{IndexId, IndexUid, NodeId, ShardId, SourceId, SourceUid}; use serde::Serialize; -pub(super) use shard_table::{NextShardId, ScalingMode, ShardEntry, ShardStats, ShardTable}; +pub(super) use shard_table::{ScalingMode, ShardEntry, ShardStats, ShardTable}; use tracing::{info, instrument, warn}; /// The control plane maintains a model in sync with the metastore. @@ -127,7 +127,6 @@ impl ControlPlaneModel { index_uid, source_id, shards, - next_shard_id, } = list_shards_subresponse; let source_uid = SourceUid { index_uid: IndexUid::parse(&index_uid).map_err(|invalid_index_uri| { @@ -138,7 +137,7 @@ impl ControlPlaneModel { source_id, }; self.shard_table - .initialize_source_shards(source_uid, shards, next_shard_id); + .initialize_source_shards(source_uid, shards); } } info!( @@ -280,20 +279,15 @@ impl ControlPlaneModel { self.shard_table.list_shards(source_uid) } - pub fn next_shard_id(&self, source_uid: &SourceUid) -> Option { - self.shard_table.next_shard_id(source_uid) - } - /// Inserts the shards that have just been opened by calling `open_shards` on the metastore. pub fn insert_newly_opened_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, - shards: Vec, - next_shard_id: NextShardId, + opened_shards: Vec, ) { self.shard_table - .insert_newly_opened_shards(index_uid, source_id, shards, next_shard_id); + .insert_newly_opened_shards(index_uid, source_id, opened_shards); } /// Finds open shards for a given index and source and whose leaders are not in the set of @@ -303,7 +297,7 @@ impl ControlPlaneModel { index_uid: &IndexUid, source_id: &SourceId, unavailable_leaders: &FnvHashSet, - ) -> Option<(Vec, NextShardId)> { + ) -> Option> { self.shard_table .find_open_shards(index_uid, source_id, unavailable_leaders) } @@ -405,20 +399,18 @@ mod tests { index_uid: "test-index-0:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), shards: vec![Shard { - shard_id: 42, + shard_id: Some(ShardId::from(42)), index_uid: "test-index-0:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), shard_state: ShardState::Open as i32, leader_id: "node1".to_string(), ..Default::default() }], - next_shard_id: 43, }, metastore::ListShardsSubresponse { index_uid: "test-index-1:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), shards: Vec::new(), - next_shard_id: 1, }, ]; let response = metastore::ListShardsResponse { subresponses }; @@ -457,10 +449,7 @@ mod tests { .unwrap() .collect(); assert_eq!(shards.len(), 1); - assert_eq!(shards[0].shard_id, 42); - - let next_shard_id = model.next_shard_id(&source_uid_0).unwrap(); - assert_eq!(next_shard_id, 43); + assert_eq!(shards[0].shard_id(), ShardId::from(42)); let source_uid_1 = SourceUid { index_uid: "test-index-1:0".into(), @@ -472,9 +461,6 @@ mod tests { .unwrap() .collect(); assert_eq!(shards.len(), 0); - - let next_shard_id = model.next_shard_id(&source_uid_1).unwrap(); - assert_eq!(next_shard_id, 1); } #[test] diff --git a/quickwit/quickwit-control-plane/src/model/shard_table.rs b/quickwit/quickwit-control-plane/src/model/shard_table.rs index 8ea8d8c6c76..14e2b522945 100644 --- a/quickwit/quickwit-control-plane/src/model/shard_table.rs +++ b/quickwit/quickwit-control-plane/src/model/shard_table.rs @@ -50,8 +50,6 @@ pub(crate) enum ScalingMode { Down, } -pub(crate) type NextShardId = ShardId; - #[derive(Debug, Clone)] pub(crate) struct ShardEntry { pub shard: Shard, @@ -84,7 +82,6 @@ impl From for ShardEntry { #[derive(Debug)] pub(crate) struct ShardTableEntry { shard_entries: FnvHashMap, - next_shard_id: NextShardId, scaling_up_rate_limiter: RateLimiter, scaling_down_rate_limiter: RateLimiter, } @@ -93,7 +90,6 @@ impl Default for ShardTableEntry { fn default() -> Self { Self { shard_entries: Default::default(), - next_shard_id: Self::DEFAULT_NEXT_SHARD_ID, scaling_up_rate_limiter: RateLimiter::from_settings(SCALING_UP_RATE_LIMITER_SETTINGS), scaling_down_rate_limiter: RateLimiter::from_settings( SCALING_DOWN_RATE_LIMITER_SETTINGS, @@ -103,20 +99,17 @@ impl Default for ShardTableEntry { } impl ShardTableEntry { - const DEFAULT_NEXT_SHARD_ID: NextShardId = 1; // `1` matches the PostgreSQL sequence min value. - - pub fn from_shards(shards: Vec, next_shard_id: NextShardId) -> Self { + pub fn from_shards(shards: Vec) -> Self { let shard_entries = shards .into_iter() .filter(|shard| { let shard_state = shard.shard_state(); shard_state == ShardState::Open || shard_state == ShardState::Closed }) - .map(|shard| (shard.shard_id, shard.into())) + .map(|shard| (shard.shard_id().clone(), shard.into())) .collect(); Self { shard_entries, - next_shard_id, ..Default::default() } } @@ -124,10 +117,6 @@ impl ShardTableEntry { fn is_empty(&self) -> bool { self.shard_entries.is_empty() } - - fn is_default(&self) -> bool { - self.is_empty() && self.next_shard_id == Self::DEFAULT_NEXT_SHARD_ID - } } // A table that keeps track of the existing shards for each index and source, @@ -153,7 +142,7 @@ fn remove_shard_from_ingesters_internal( .get_mut(&node) .expect("shard table reached inconsistent state"); let shard_ids = ingester_shards.get_mut(source_uid).unwrap(); - shard_ids.remove(&shard.shard_id); + shard_ids.remove(shard.shard_id()); } } @@ -181,6 +170,7 @@ impl ShardTable { /// Checks whether the shard table is consistent. /// /// Panics if it is not. + #[allow(clippy::mutable_key_type)] fn check_invariant(&self) { // This function is expensive! Let's not call it in release mode. if !cfg!(debug_assertions) { @@ -189,7 +179,7 @@ impl ShardTable { let mut shard_sets_in_shard_table = FnvHashSet::default(); for (source_uid, shard_table_entry) in &self.table_entries { for (shard_id, shard_entry) in &shard_table_entry.shard_entries { - debug_assert_eq!(shard_id, &shard_entry.shard.shard_id); + debug_assert_eq!(shard_id, shard_entry.shard.shard_id()); debug_assert_eq!(source_uid.index_uid.as_str(), &shard_entry.shard.index_uid); for node in shard_entry.shard.ingester_nodes() { shard_sets_in_shard_table.insert((node, source_uid, shard_id)); @@ -248,7 +238,7 @@ impl ShardTable { let table_entry = ShardTableEntry::default(); let previous_table_entry_opt = self.table_entries.insert(source_uid, table_entry); if let Some(previous_table_entry) = previous_table_entry_opt { - if !previous_table_entry.is_default() { + if !previous_table_entry.is_empty() { error!( "shard table entry for index `{}` and source `{}` already exists", index_uid.index_id(), @@ -305,19 +295,12 @@ impl ShardTable { .map(|table_entry| table_entry.shard_entries.values()) } - pub fn next_shard_id(&self, source_uid: &SourceUid) -> Option { - self.table_entries - .get(source_uid) - .map(|table_entry| table_entry.next_shard_id) - } - /// Updates the shard table. pub fn insert_newly_opened_shards( &mut self, index_uid: &IndexUid, source_id: &SourceId, opened_shards: Vec, - next_shard_id: NextShardId, ) { let source_uid = SourceUid { index_uid: index_uid.clone(), @@ -337,7 +320,7 @@ impl ShardTable { for node in shard.ingester_nodes() { let ingester_shards = self.ingester_shards.entry(node).or_default(); let shard_ids = ingester_shards.entry(source_uid.clone()).or_default(); - shard_ids.insert(shard.shard_id); + shard_ids.insert(shard.shard_id().clone()); } } match self.table_entries.entry(source_uid) { @@ -349,10 +332,9 @@ impl ShardTable { // knows more about the state of the shards than the metastore. table_entry .shard_entries - .entry(opened_shard.shard_id) + .entry(opened_shard.shard_id().clone()) .or_insert(opened_shard.into()); } - table_entry.next_shard_id = next_shard_id; } // This should never happen if the control plane view is consistent with the state of // the metastore, so should we panic here? Warnings are most likely going to go @@ -360,11 +342,10 @@ impl ShardTable { Entry::Vacant(entry) => { let shard_entries: FnvHashMap = opened_shards .into_iter() - .map(|shard| (shard.shard_id, shard.into())) + .map(|shard| (shard.shard_id().clone(), shard.into())) .collect(); let table_entry = ShardTableEntry { shard_entries, - next_shard_id, ..Default::default() }; entry.insert(table_entry); @@ -380,7 +361,7 @@ impl ShardTable { index_uid: &IndexUid, source_id: &SourceId, unavailable_leaders: &FnvHashSet, - ) -> Option<(Vec, NextShardId)> { + ) -> Option> { let source_uid = SourceUid { index_uid: index_uid.clone(), source_id: source_id.clone(), @@ -394,8 +375,7 @@ impl ShardTable { }) .cloned() .collect(); - - Some((open_shards, table_entry.next_shard_id)) + Some(open_shards) } pub fn update_shards( @@ -452,7 +432,7 @@ impl ShardTable { if let Some(shard_entry) = table_entry.shard_entries.get_mut(shard_id) { if !shard_entry.is_closed() { shard_entry.set_shard_state(ShardState::Closed); - closed_shard_ids.push(*shard_id); + closed_shard_ids.push(shard_id.clone()); } } } @@ -468,7 +448,7 @@ impl ShardTable { if let Some(shard_entry) = table_entry.shard_entries.remove(shard_id) { shard_entries_to_remove.push(shard_entry); } else { - warn!(shard = *shard_id, "deleting a non-existing shard"); + warn!(shard=%shard_id, "deleting a non-existing shard"); } } } @@ -484,20 +464,15 @@ impl ShardTable { /// Set the shards for a given source. /// This function panics if an entry was previously associated to the source uid. - pub(crate) fn initialize_source_shards( - &mut self, - source_uid: SourceUid, - shards: Vec, - next_shard_id: NextShardId, - ) { + pub(crate) fn initialize_source_shards(&mut self, source_uid: SourceUid, shards: Vec) { for shard in &shards { for node in shard.ingester_nodes() { let ingester_shards = self.ingester_shards.entry(node).or_default(); let shard_ids = ingester_shards.entry(source_uid.clone()).or_default(); - shard_ids.insert(shard.shard_id); + shard_ids.insert(shard.shard_id().clone()); } } - let table_entry = ShardTableEntry::from_shards(shards, next_shard_id); + let table_entry = ShardTableEntry::from_shards(shards); let previous_entry = self.table_entries.insert(source_uid, table_entry); assert!(previous_entry.is_none()); self.check_invariant(); @@ -553,7 +528,7 @@ mod tests { self.shard_entries .values() .map(|shard_entry| shard_entry.shard.clone()) - .sorted_unstable_by_key(|shard| shard.shard_id) + .sorted_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)) .collect() } } @@ -564,11 +539,13 @@ mod tests { index_uid: &IndexUid, source_id: &SourceId, unavailable_leaders: &FnvHashSet, - ) -> Option<(Vec, NextShardId)> { + ) -> Option> { self.find_open_shards(index_uid, source_id, unavailable_leaders) - .map(|(mut shards, next_shard_id)| { - shards.sort_by_key(|shard_entry| shard_entry.shard.shard_id); - (shards, next_shard_id) + .map(|mut shards| { + shards.sort_unstable_by(|left, right| { + left.shard.shard_id.cmp(&right.shard.shard_id) + }); + shards }) } } @@ -612,7 +589,6 @@ mod tests { }; let table_entry = shard_table.table_entries.get(&source_uid).unwrap(); assert!(table_entry.shard_entries.is_empty()); - assert_eq!(table_entry.next_shard_id, 1); } #[test] @@ -634,12 +610,12 @@ mod tests { let shard_01 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() }; - shard_table.insert_newly_opened_shards(&index_uid, &source_id, vec![shard_01], 2); + shard_table.insert_newly_opened_shards(&index_uid, &source_id, vec![shard_01]); let shards = shard_table.list_shards(&source_uid).unwrap(); assert_eq!(shards.count(), 1); @@ -655,12 +631,12 @@ mod tests { let shard_01 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.insert_newly_opened_shards(&index_uid_0, &source_id, vec![shard_01.clone()], 2); + shard_table.insert_newly_opened_shards(&index_uid_0, &source_id, vec![shard_01.clone()]); assert_eq!(shard_table.table_entries.len(), 1); @@ -672,21 +648,20 @@ mod tests { let shards = table_entry.shards(); assert_eq!(shards.len(), 1); assert_eq!(shards[0], shard_01); - assert_eq!(table_entry.next_shard_id, 2); shard_table .table_entries .get_mut(&source_uid) .unwrap() .shard_entries - .get_mut(&1) + .get_mut(&ShardId::from(1)) .unwrap() .set_shard_state(ShardState::Unavailable); let shard_02 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -696,7 +671,6 @@ mod tests { &index_uid_0, &source_id, vec![shard_01.clone(), shard_02.clone()], - 3, ); assert_eq!(shard_table.table_entries.len(), 1); @@ -710,7 +684,6 @@ mod tests { assert_eq!(shards.len(), 2); assert_eq!(shards[0].shard_state(), ShardState::Unavailable); assert_eq!(shards[1], shard_02); - assert_eq!(table_entry.next_shard_id, 3); } #[test] @@ -723,16 +696,15 @@ mod tests { let mut unavailable_ingesters = FnvHashSet::default(); - let (open_shards, next_shard_id) = shard_table + let open_shards = shard_table .find_open_shards_sorted(&index_uid, &source_id, &unavailable_ingesters) .unwrap(); assert_eq!(open_shards.len(), 0); - assert_eq!(next_shard_id, 1); let shard_01 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() @@ -740,7 +712,7 @@ mod tests { let shard_02 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Unavailable as i32, ..Default::default() @@ -748,7 +720,7 @@ mod tests { let shard_03 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 3, + shard_id: Some(ShardId::from(3)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -756,7 +728,7 @@ mod tests { let shard_04 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 4, + shard_id: Some(ShardId::from(4)), leader_id: "test-leader-1".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -765,24 +737,21 @@ mod tests { &index_uid, &source_id, vec![shard_01, shard_02, shard_03.clone(), shard_04.clone()], - 5, ); - let (open_shards, next_shard_id) = shard_table + let open_shards = shard_table .find_open_shards_sorted(&index_uid, &source_id, &unavailable_ingesters) .unwrap(); assert_eq!(open_shards.len(), 2); assert_eq!(open_shards[0].shard, shard_03); assert_eq!(open_shards[1].shard, shard_04); - assert_eq!(next_shard_id, 5); unavailable_ingesters.insert("test-leader-0".into()); - let (open_shards, next_shard_id) = shard_table + let open_shards = shard_table .find_open_shards_sorted(&index_uid, &source_id, &unavailable_ingesters) .unwrap(); assert_eq!(open_shards.len(), 1); assert_eq!(open_shards[0].shard, shard_04); - assert_eq!(next_shard_id, 5); } #[test] @@ -795,28 +764,28 @@ mod tests { let shard_01 = Shard { index_uid: index_uid.to_string(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, ..Default::default() }; let shard_02 = Shard { index_uid: index_uid.to_string(), source_id: source_id.clone(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, ..Default::default() }; let shard_03 = Shard { index_uid: index_uid.to_string(), source_id: source_id.clone(), - shard_id: 3, + shard_id: Some(ShardId::from(3)), shard_state: ShardState::Unavailable as i32, ..Default::default() }; let shard_04 = Shard { index_uid: index_uid.to_string(), source_id: source_id.clone(), - shard_id: 4, + shard_id: Some(ShardId::from(4)), shard_state: ShardState::Open as i32, ..Default::default() }; @@ -824,7 +793,6 @@ mod tests { &index_uid, &source_id, vec![shard_01, shard_02, shard_03, shard_04], - 5, ); let source_uid = SourceUid { index_uid, @@ -832,27 +800,27 @@ mod tests { }; let shard_infos = BTreeSet::from_iter([ ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(1), }, ShardInfo { - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(2), }, ShardInfo { - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(3), }, ShardInfo { - shard_id: 4, + shard_id: ShardId::from(4), shard_state: ShardState::Closed, ingestion_rate: RateMibPerSec(4), }, ShardInfo { - shard_id: 5, + shard_id: ShardId::from(5), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(5), }, @@ -865,26 +833,26 @@ mod tests { .list_shards(&source_uid) .unwrap() .cloned() - .sorted_by_key(|shard_entry| shard_entry.shard.shard_id) + .sorted_unstable_by(|left, right| left.shard.shard_id.cmp(&right.shard.shard_id)) .collect(); assert_eq!(shard_entries.len(), 4); - assert_eq!(shard_entries[0].shard.shard_id, 1); + assert_eq!(shard_entries[0].shard.shard_id(), ShardId::from(1)); assert_eq!(shard_entries[0].shard.shard_state(), ShardState::Open); assert_eq!(shard_entries[0].ingestion_rate, RateMibPerSec(1)); - assert_eq!(shard_entries[1].shard.shard_id, 2); + assert_eq!(shard_entries[1].shard.shard_id(), ShardId::from(2)); assert_eq!(shard_entries[1].shard.shard_state(), ShardState::Open); assert_eq!(shard_entries[1].ingestion_rate, RateMibPerSec(2)); - assert_eq!(shard_entries[2].shard.shard_id, 3); + assert_eq!(shard_entries[2].shard.shard_id(), ShardId::from(3)); assert_eq!( shard_entries[2].shard.shard_state(), ShardState::Unavailable ); assert_eq!(shard_entries[2].ingestion_rate, RateMibPerSec(3)); - assert_eq!(shard_entries[3].shard.shard_id, 4); + assert_eq!(shard_entries[3].shard.shard_id(), ShardId::from(4)); assert_eq!(shard_entries[3].shard.shard_state(), ShardState::Closed); assert_eq!(shard_entries[3].ingestion_rate, RateMibPerSec(4)); } @@ -900,7 +868,7 @@ mod tests { let shard_01 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -908,7 +876,7 @@ mod tests { let shard_02 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Closed as i32, ..Default::default() @@ -916,25 +884,23 @@ mod tests { let shard_11 = Shard { index_uid: index_uid_1.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() }; - shard_table.insert_newly_opened_shards( - &index_uid_0, - &source_id, - vec![shard_01, shard_02], - 3, - ); - shard_table.insert_newly_opened_shards(&index_uid_1, &source_id, vec![shard_11], 2); + shard_table.insert_newly_opened_shards(&index_uid_0, &source_id, vec![shard_01, shard_02]); + shard_table.insert_newly_opened_shards(&index_uid_1, &source_id, vec![shard_11]); let source_uid_0 = SourceUid { index_uid: index_uid_0, source_id, }; - let closed_shard_ids = shard_table.close_shards(&source_uid_0, &[1, 2, 3]); - assert_eq!(closed_shard_ids, &[1]); + let closed_shard_ids = shard_table.close_shards( + &source_uid_0, + &[ShardId::from(1), ShardId::from(2), ShardId::from(3)], + ); + assert_eq!(closed_shard_ids, &[ShardId::from(1)]); let table_entry = shard_table.table_entries.get(&source_uid_0).unwrap(); let shards = table_entry.shards(); @@ -952,7 +918,7 @@ mod tests { let shard_01 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -960,7 +926,7 @@ mod tests { let shard_02 = Shard { index_uid: index_uid_0.clone().into(), source_id: source_id.clone(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -968,7 +934,7 @@ mod tests { let shard_11 = Shard { index_uid: index_uid_1.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), leader_id: "test-leader-0".to_string(), shard_state: ShardState::Open as i32, ..Default::default() @@ -977,21 +943,20 @@ mod tests { &index_uid_0, &source_id, vec![shard_01.clone(), shard_02], - 3, ); - shard_table.insert_newly_opened_shards(&index_uid_1, &source_id, vec![shard_11], 2); + shard_table.insert_newly_opened_shards(&index_uid_1, &source_id, vec![shard_11]); let source_uid_0 = SourceUid { index_uid: index_uid_0.clone(), source_id: source_id.clone(), }; - shard_table.delete_shards(&source_uid_0, &[2]); + shard_table.delete_shards(&source_uid_0, &[ShardId::from(2)]); let source_uid_1 = SourceUid { index_uid: index_uid_1.clone(), source_id: source_id.clone(), }; - shard_table.delete_shards(&source_uid_1, &[1]); + shard_table.delete_shards(&source_uid_1, &[ShardId::from(1)]); assert_eq!(shard_table.table_entries.len(), 2); @@ -999,11 +964,9 @@ mod tests { let shards = table_entry.shards(); assert_eq!(shards.len(), 1); assert_eq!(shards[0], shard_01); - assert_eq!(table_entry.next_shard_id, 3); let table_entry = shard_table.table_entries.get(&source_uid_1).unwrap(); assert!(table_entry.is_empty()); - assert_eq!(table_entry.next_shard_id, 2); } #[test] diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 6f449d5845d..a74315772ca 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -523,7 +523,7 @@ impl IndexingService { continue; }; let assignment = Assignment { - shard_ids: task.shard_ids.iter().copied().collect(), + shard_ids: task.shard_ids.iter().cloned().collect(), }; let message = AssignShards(assignment); @@ -665,7 +665,7 @@ impl IndexingService { .last_observation() .shard_ids .iter() - .copied() + .cloned() .collect(), }) .collect(); diff --git a/quickwit/quickwit-indexing/src/models/shard_positions.rs b/quickwit/quickwit-indexing/src/models/shard_positions.rs index 0b992d3f1b1..4e76b7e10da 100644 --- a/quickwit/quickwit-indexing/src/models/shard_positions.rs +++ b/quickwit/quickwit-indexing/src/models/shard_positions.rs @@ -229,7 +229,7 @@ impl ShardPositionsService { }; let shard_positions: Vec<(ShardId, Position)> = shard_positions_map .iter() - .map(|(&shard_id, position)| (shard_id, position.clone())) + .map(|(shard_id, position)| (shard_id.clone(), position.clone())) .collect(); self.event_broker.publish(ShardPositionsUpdate { source_uid, @@ -350,27 +350,30 @@ mod tests { event_broker1.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, Position::Beginning)], + vec![(ShardId::from(1), Position::Beginning)], )); event_broker1.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, Position::offset(10u64))], + vec![(ShardId::from(2), Position::offset(10u64))], )); event_broker1.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, Position::offset(10u64))], + vec![(ShardId::from(1), Position::offset(10u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, Position::offset(10u64))], + vec![(ShardId::from(2), Position::offset(10u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, Position::offset(12u64))], + vec![(ShardId::from(2), Position::offset(12u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, Position::Beginning), (2, Position::offset(12u64))], + vec![ + (ShardId::from(1), Position::Beginning), + (ShardId::from(2), Position::offset(12u64)), + ], )); let mut updates1: Vec> = Vec::new(); @@ -384,10 +387,19 @@ mod tests { assert_eq!( updates1, vec![ - vec![(1, Position::Beginning)], - vec![(1, Position::Beginning), (2, Position::offset(10u64))], - vec![(1, Position::offset(10u64)), (2, Position::offset(10u64)),], - vec![(1, Position::offset(10u64)), (2, Position::offset(12u64)),], + vec![(ShardId::from(1), Position::Beginning)], + vec![ + (ShardId::from(1), Position::Beginning), + (ShardId::from(2), Position::offset(10u64)) + ], + vec![ + (ShardId::from(1), Position::offset(10u64)), + (ShardId::from(2), Position::offset(10u64)), + ], + vec![ + (ShardId::from(1), Position::offset(10u64)), + (ShardId::from(2), Position::offset(12u64)), + ], ] ); @@ -401,10 +413,16 @@ mod tests { assert_eq!( updates2, vec![ - vec![(2, Position::offset(10u64))], - vec![(2, Position::offset(12u64))], - vec![(1, Position::Beginning), (2, Position::offset(12u64))], - vec![(1, Position::offset(10u64)), (2, Position::offset(12u64))], + vec![(ShardId::from(2), Position::offset(10u64))], + vec![(ShardId::from(2), Position::offset(12u64))], + vec![ + (ShardId::from(1), Position::Beginning), + (ShardId::from(2), Position::offset(12u64)) + ], + vec![ + (ShardId::from(1), Position::offset(10u64)), + (ShardId::from(2), Position::offset(12u64)) + ], ] ); @@ -442,33 +460,33 @@ mod tests { { event_broker.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, Position::Beginning)], + vec![(ShardId::from(1), Position::Beginning)], )); tokio::time::sleep(Duration::from_secs(1)).await; let value = cluster.get_self_key_value(&key).await.unwrap(); - assert_eq!(&value, r#"{"1":""}"#); + assert_eq!(&value, r#"{"00000000000000000001":""}"#); } { event_broker.publish(LocalShardPositionsUpdate::new( source_uid.clone(), vec![ - (1, Position::offset(1_000u64)), - (2, Position::offset(2_000u64)), + (ShardId::from(1), Position::offset(1_000u64)), + (ShardId::from(2), Position::offset(2_000u64)), ], )); tokio::time::sleep(Duration::from_secs(1)).await; let value = cluster.get_self_key_value(&key).await.unwrap(); assert_eq!( &value, - r#"{"1":"00000000000000001000","2":"00000000000000002000"}"# + r#"{"00000000000000000001":"00000000000000001000","00000000000000000002":"00000000000000002000"}"# ); } { event_broker.publish(LocalShardPositionsUpdate::new( source_uid.clone(), vec![ - (1, Position::offset(999u64)), - (3, Position::offset(3_000u64)), + (ShardId::from(1), Position::offset(999u64)), + (ShardId::from(3), Position::offset(3_000u64)), ], )); tokio::time::sleep(Duration::from_secs(1)).await; @@ -476,7 +494,7 @@ mod tests { // We do not update the position that got lower, nor the position that disappeared assert_eq!( &value, - r#"{"1":"00000000000000001000","2":"00000000000000002000","3":"00000000000000003000"}"# + r#"{"00000000000000000001":"00000000000000001000","00000000000000000002":"00000000000000002000","00000000000000000003":"00000000000000003000"}"# ); } universe.assert_quit().await; diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 95de09c2abf..bd9cc9dd047 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -214,14 +214,14 @@ impl IngestSource { }; let assigned_shard = self .assigned_shards - .get_mut(&fetch_payload.shard_id) + .get_mut(fetch_payload.shard_id()) .expect("shard should be assigned"); assigned_shard.status = IndexingStatus::Active; let partition_id = assigned_shard.partition_id.clone(); - let from_position_exclusive = fetch_payload.from_position_exclusive(); - let to_position_inclusive = fetch_payload.to_position_inclusive(); + let from_position_exclusive = fetch_payload.from_position_exclusive().clone(); + let to_position_inclusive = fetch_payload.to_position_inclusive().clone(); for mrecord in decoded_mrecords(mrecord_batch) { match mrecord { @@ -252,14 +252,14 @@ impl IngestSource { ) -> anyhow::Result<()> { let assigned_shard = self .assigned_shards - .get_mut(&fetch_eof.shard_id) + .get_mut(fetch_eof.shard_id()) .expect("shard should be assigned"); assigned_shard.status = IndexingStatus::ReachedEof; let partition_id = assigned_shard.partition_id.clone(); let from_position_exclusive = assigned_shard.current_position_inclusive.clone(); - let to_position_inclusive = fetch_eof.eof_position(); + let to_position_inclusive = fetch_eof.eof_position().clone(); batch_builder .checkpoint_delta @@ -335,7 +335,7 @@ impl IngestSource { let truncate_shards_subrequest = TruncateShardsSubrequest { index_uid: self.client_id.source_uid.index_uid.clone().into(), source_id: self.client_id.source_uid.source_id.clone(), - shard_id, + shard_id: Some(shard_id), truncate_up_to_position_inclusive: Some(truncate_up_to_position_inclusive), }; if let Some(follower_id) = &shard.follower_id_opt { @@ -415,7 +415,7 @@ impl IngestSource { || self .assigned_shards .keys() - .copied() + .cloned() .filter(|shard_id| !new_assigned_shard_ids.contains(shard_id)) .any(|removed_shard_id| { let Some(assigned_shard) = self.assigned_shards.get(&removed_shard_id) else { @@ -572,12 +572,12 @@ impl Source for IngestSource { Vec::with_capacity(acquire_shards_subresponse.acquired_shards.len()); for acquired_shard in acquire_shards_subresponse.acquired_shards { + let shard_id = acquired_shard.shard_id().clone(); let leader_id: NodeId = acquired_shard.leader_id.into(); let follower_id_opt: Option = acquired_shard.follower_id.map(Into::into); let index_uid: IndexUid = acquired_shard.index_uid.into(); let source_id: SourceId = acquired_shard.source_id; - let shard_id = acquired_shard.shard_id; - let partition_id = PartitionId::from(shard_id); + let partition_id = PartitionId::from(shard_id.as_str()); let mut current_position_inclusive = acquired_shard .publish_position_inclusive .unwrap_or_default(); @@ -591,7 +591,7 @@ impl Source for IngestSource { follower_id_opt.clone(), index_uid, source_id, - shard_id, + shard_id.clone(), from_position_exclusive, )) .await @@ -607,7 +607,7 @@ impl Source for IngestSource { } else { IndexingStatus::Active }; - truncate_up_to_positions.push((shard_id, current_position_inclusive.clone())); + truncate_up_to_positions.push((shard_id.clone(), current_position_inclusive.clone())); let assigned_shard = AssignedShard { leader_id, @@ -632,7 +632,7 @@ impl Source for IngestSource { Vec::with_capacity(checkpoint.num_partitions()); for (partition_id, position) in checkpoint.iter() { - let shard_id = partition_id.as_u64().expect("shard ID should be a u64"); + let shard_id = ShardId::from(partition_id.as_str()); truncate_up_to_positions.push((shard_id, position)); } self.truncate(truncate_up_to_positions).await; @@ -710,7 +710,7 @@ mod tests { .expect_acquire_shards() .withf(|request| { assert_eq!(request.subrequests.len(), 1); - request.subrequests[0].shard_ids == [0] + request.subrequests[0].shard_ids == [ShardId::from(0)] }) .once() .returning(|request| { @@ -726,7 +726,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::offset(10u64)), publish_token: Some(publish_token.to_string()), @@ -740,7 +740,7 @@ mod tests { .once() .withf(|request| { assert_eq!(request.subrequests.len(), 1); - request.subrequests[0].shard_ids == [1] + request.subrequests[0].shard_ids == [ShardId::from(1)] }) .returning(|request| { assert_eq!(request.subrequests.len(), 1); @@ -758,7 +758,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), @@ -771,7 +771,7 @@ mod tests { .expect_acquire_shards() .withf(|request| { assert_eq!(request.subrequests.len(), 1); - request.subrequests[0].shard_ids == [1, 2] + request.subrequests[0].shard_ids == [ShardId::from(1), ShardId::from(2)] }) .once() .returning(|request| { @@ -791,7 +791,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), @@ -801,7 +801,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::offset(12u64)), publish_token: Some(publish_token.to_string()), @@ -823,7 +823,7 @@ mod tests { .expect_open_fetch_stream() .withf(|request| { request.from_position_exclusive() == Position::offset(10u64) - && request.shard_id == 0 + && request.shard_id() == ShardId::from(0) }) .once() .returning(move |request| { @@ -843,7 +843,7 @@ mod tests { .expect_open_fetch_stream() .withf(|request| { request.from_position_exclusive() == Position::offset(11u64) - && request.shard_id == 1 + && request.shard_id() == ShardId::from(1) }) .times(2) .returning(move |request| { @@ -863,7 +863,7 @@ mod tests { .expect_open_fetch_stream() .withf(|request| { request.from_position_exclusive() == Position::offset(12u64) - && request.shard_id == 2 + && request.shard_id() == ShardId::from(2) }) .once() .returning(move |request| { @@ -880,7 +880,7 @@ mod tests { }); ingester_mock_0 .expect_truncate_shards() - .withf(|truncate_req| truncate_req.subrequests[0].shard_id == 0) + .withf(|truncate_req| truncate_req.subrequests[0].shard_id() == ShardId::from(0)) .once() .returning(|request| { assert_eq!(request.ingester_id, "test-ingester-0"); @@ -900,7 +900,7 @@ mod tests { ingester_mock_0 .expect_truncate_shards() - .withf(|truncate_req| truncate_req.subrequests[0].shard_id == 1) + .withf(|truncate_req| truncate_req.subrequests[0].shard_id() == ShardId::from(1)) .once() .returning(|request| { assert_eq!(request.ingester_id, "test-ingester-0"); @@ -920,8 +920,8 @@ mod tests { .expect_truncate_shards() .withf(|truncate_req| { truncate_req.subrequests.len() == 2 - && truncate_req.subrequests[0].shard_id == 1 - && truncate_req.subrequests[1].shard_id == 2 + && truncate_req.subrequests[0].shard_id() == ShardId::from(1) + && truncate_req.subrequests[1].shard_id() == ShardId::from(2) }) .once() .returning(|request| { @@ -979,7 +979,7 @@ mod tests { // We assign [0] (previously []). // The stream does not need to be reset. - let shard_ids: BTreeSet = once(0).collect(); + let shard_ids: BTreeSet = once(0).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -993,7 +993,7 @@ mod tests { // We assign [0,1] (previously [0]). This should just add the shard 1. // The stream does not need to be reset. - let shard_ids: BTreeSet = (0..2).collect(); + let shard_ids: BTreeSet = (0..2).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -1006,7 +1006,7 @@ mod tests { // We assign [1,2]. (previously [0,1]) This should reset the stream // because the shard 0 has to be removed. // The publish lock should be killed and a new one should be created. - let shard_ids: BTreeSet = (1..3).collect(); + let shard_ids: BTreeSet = (1..3).map(ShardId::from).collect(); let publish_lock = source.publish_lock.clone(); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -1035,7 +1035,7 @@ mod tests { assert_eq!(source.assigned_shards.len(), 2); - let assigned_shard = source.assigned_shards.get(&1).unwrap(); + let assigned_shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); let expected_assigned_shard = AssignedShard { leader_id: "test-ingester-0".into(), follower_id_opt: None, @@ -1045,7 +1045,7 @@ mod tests { }; assert_eq!(assigned_shard, &expected_assigned_shard); - let assigned_shard = source.assigned_shards.get(&2).unwrap(); + let assigned_shard = source.assigned_shards.get(&ShardId::from(2)).unwrap(); let expected_assigned_shard = AssignedShard { leader_id: "test-ingester-0".into(), follower_id_opt: None, @@ -1085,7 +1085,7 @@ mod tests { let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, vec![1, 2]); + assert_eq!(subrequest.shard_ids, [ShardId::from(1), ShardId::from(2)]); let response = AcquireShardsResponse { subresponses: vec![AcquireShardsSubresponse { @@ -1097,7 +1097,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::eof(11u64)), publish_token: Some(publish_token.to_string()), @@ -1107,7 +1107,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::Beginning.as_eof()), publish_token: Some(publish_token.to_string()), @@ -1130,7 +1130,7 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.index_uid, "test-index:0"); assert_eq!(subrequest_0.source_id, "test-source"); - assert_eq!(subrequest_0.shard_id, 1); + assert_eq!(subrequest_0.shard_id(), ShardId::from(1)); assert_eq!( subrequest_0.truncate_up_to_position_inclusive(), Position::eof(11u64) @@ -1139,7 +1139,7 @@ mod tests { let subrequest_1 = &request.subrequests[1]; assert_eq!(subrequest_1.index_uid, "test-index:0"); assert_eq!(subrequest_1.source_id, "test-source"); - assert_eq!(subrequest_1.shard_id, 2); + assert_eq!(subrequest_1.shard_id(), ShardId::from(2)); assert_eq!( subrequest_1.truncate_up_to_position_inclusive(), Position::Beginning.as_eof() @@ -1184,7 +1184,8 @@ mod tests { ActorContext::for_test(&universe, source_mailbox, observable_state_tx); // In this scenario, the indexer will be able to acquire shard 1 and 2. - let shard_ids: BTreeSet = BTreeSet::from_iter([1, 2]); + let shard_ids: BTreeSet = + BTreeSet::from_iter([ShardId::from(1), ShardId::from(2)]); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -1196,7 +1197,10 @@ mod tests { index_uid: IndexUid::parse("test-index:0").unwrap(), source_id: "test-source".to_string(), }, - vec![(1, Position::eof(11u64)), (2, Position::Beginning.as_eof())], + vec![ + (ShardId::from(1), Position::eof(11u64)), + (ShardId::from(2), Position::Beginning.as_eof()), + ], ); let local_update = shard_positions_update_rx.recv().await.unwrap(); assert_eq!(local_update, expected_local_update); @@ -1228,7 +1232,7 @@ mod tests { let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, vec![1, 2]); + assert_eq!(subrequest.shard_ids, [ShardId::from(1), ShardId::from(2)]); let response = AcquireShardsResponse { subresponses: vec![AcquireShardsSubresponse { @@ -1240,7 +1244,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), @@ -1250,7 +1254,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Closed as i32, publish_position_inclusive: Some(Position::eof(22u64)), publish_token: Some(publish_token.to_string()), @@ -1273,7 +1277,7 @@ mod tests { ); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(11u64)); let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); @@ -1287,12 +1291,12 @@ mod tests { assert_eq!(request.subrequests.len(), 2); request .subrequests - .sort_by_key(|subrequest| subrequest.shard_id); + .sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.truncate_up_to_position_inclusive(), Position::offset(11u64) @@ -1301,7 +1305,7 @@ mod tests { let subrequest = &request.subrequests[1]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 2); + assert_eq!(subrequest.shard_id(), ShardId::from(2)); assert_eq!( subrequest.truncate_up_to_position_inclusive(), Position::eof(22u64) @@ -1346,7 +1350,7 @@ mod tests { ActorContext::for_test(&universe, source_mailbox, observable_state_tx); // In this scenario, the indexer will only be able to acquire shard 1. - let shard_ids: BTreeSet = (1..3).collect(); + let shard_ids: BTreeSet = (1..3).map(ShardId::from).collect(); assert_eq!( shard_positions_update_rx.try_recv().unwrap_err(), TryRecvError::Empty @@ -1364,7 +1368,10 @@ mod tests { index_uid: IndexUid::parse("test-index:0").unwrap(), source_id: "test-source".to_string(), }, - vec![(1, Position::offset(11u64)), (2, Position::eof(22u64))], + vec![ + (ShardId::from(1), Position::offset(11u64)), + (ShardId::from(2), Position::eof(22u64)), + ], ); assert_eq!( local_shard_positions_update, @@ -1409,7 +1416,7 @@ mod tests { // In this scenario, the ingester receives fetch responses from shard 1 and 2. source.assigned_shards.insert( - 1, + ShardId::from(1), AssignedShard { leader_id: "test-ingester-0".into(), follower_id_opt: None, @@ -1419,7 +1426,7 @@ mod tests { }, ); source.assigned_shards.insert( - 2, + ShardId::from(2), AssignedShard { leader_id: "test-ingester-1".into(), follower_id_opt: None, @@ -1433,7 +1440,7 @@ mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test([ "\0\0test-doc-foo", "\0\0test-doc-bar", @@ -1448,7 +1455,7 @@ mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-qux"]), from_position_exclusive: Some(Position::offset(22u64)), to_position_inclusive: Some(Position::offset(23u64)), @@ -1459,7 +1466,7 @@ mod tests { let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), eof_position: Some(Position::eof(23u64)), }; let fetch_message = FetchMessage::new_eof(fetch_eof); @@ -1498,14 +1505,14 @@ mod tests { .emit_batches(&doc_processor_mailbox, &ctx) .await .unwrap(); - let shard = source.assigned_shards.get(&2).unwrap(); + let shard = source.assigned_shards.get(&ShardId::from(2)).unwrap(); assert_eq!(shard.status, IndexingStatus::ReachedEof); fetch_message_tx .send(Err(FetchStreamError { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: ShardId::from(1), ingest_error: IngestV2Error::Internal("test-error".to_string()), })) .await @@ -1515,13 +1522,13 @@ mod tests { .emit_batches(&doc_processor_mailbox, &ctx) .await .unwrap(); - let shard = source.assigned_shards.get(&1).unwrap(); + let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::Error); let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-baz"]), from_position_exclusive: Some(Position::offset(14u64)), to_position_inclusive: Some(Position::offset(15u64)), @@ -1533,7 +1540,7 @@ mod tests { .emit_batches(&doc_processor_mailbox, &ctx) .await .unwrap(); - let shard = source.assigned_shards.get(&1).unwrap(); + let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::Active); } @@ -1559,7 +1566,7 @@ mod tests { let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, vec![1]); + assert_eq!(subrequest.shard_ids, [ShardId::from(1)]); let response = AcquireShardsResponse { subresponses: vec![AcquireShardsSubresponse { @@ -1570,7 +1577,7 @@ mod tests { follower_id: None, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, publish_position_inclusive: Some(Position::Beginning), publish_token: Some(publish_token.to_string()), @@ -1588,10 +1595,12 @@ mod tests { .returning(|request| { assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::Beginning); - Err(IngestV2Error::ShardNotFound { shard_id: 1 }) + Err(IngestV2Error::ShardNotFound { + shard_id: ShardId::from(1), + }) }); let ingester_0: IngesterServiceClient = ingester_mock_0.into(); @@ -1620,7 +1629,7 @@ mod tests { let ctx: SourceContext = ActorContext::for_test(&universe, source_mailbox, observable_state_tx); - let shard_ids: BTreeSet = BTreeSet::from_iter([1]); + let shard_ids: BTreeSet = BTreeSet::from_iter([ShardId::from(1)]); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -1632,7 +1641,7 @@ mod tests { .await .unwrap(); - let shard = source.assigned_shards.get(&1).unwrap(); + let shard = source.assigned_shards.get(&ShardId::from(1)).unwrap(); assert_eq!(shard.status, IndexingStatus::NotFound); assert_eq!( shard.current_position_inclusive, @@ -1671,21 +1680,21 @@ mod tests { assert_eq!(request.subrequests.len(), 3); let subrequest_0 = &request.subrequests[0]; - assert_eq!(subrequest_0.shard_id, 1); + assert_eq!(subrequest_0.shard_id(), ShardId::from(1)); assert_eq!( subrequest_0.truncate_up_to_position_inclusive(), Position::offset(11u64) ); let subrequest_1 = &request.subrequests[1]; - assert_eq!(subrequest_1.shard_id, 2); + assert_eq!(subrequest_1.shard_id(), ShardId::from(2)); assert_eq!( subrequest_1.truncate_up_to_position_inclusive(), Position::offset(22u64) ); let subrequest_2 = &request.subrequests[2]; - assert_eq!(subrequest_2.shard_id, 3); + assert_eq!(subrequest_2.shard_id(), ShardId::from(3)); assert_eq!( subrequest_2.truncate_up_to_position_inclusive(), Position::eof(33u64) @@ -1705,14 +1714,14 @@ mod tests { assert_eq!(request.subrequests.len(), 2); let subrequest_0 = &request.subrequests[0]; - assert_eq!(subrequest_0.shard_id, 2); + assert_eq!(subrequest_0.shard_id(), ShardId::from(2)); assert_eq!( subrequest_0.truncate_up_to_position_inclusive(), Position::offset(22u64) ); let subrequest_1 = &request.subrequests[1]; - assert_eq!(subrequest_1.shard_id, 3); + assert_eq!(subrequest_1.shard_id(), ShardId::from(3)); assert_eq!( subrequest_1.truncate_up_to_position_inclusive(), Position::eof(33u64) @@ -1732,7 +1741,7 @@ mod tests { assert_eq!(request.subrequests.len(), 1); let subrequest_0 = &request.subrequests[0]; - assert_eq!(subrequest_0.shard_id, 4); + assert_eq!(subrequest_0.shard_id(), ShardId::from(4)); assert_eq!( subrequest_0.truncate_up_to_position_inclusive(), Position::offset(44u64) @@ -1774,7 +1783,7 @@ mod tests { // In this scenario, the ingester 2 is not available and the shard 6 is no longer assigned. source.assigned_shards.insert( - 1, + ShardId::from(1), AssignedShard { leader_id: "test-ingester-0".into(), follower_id_opt: None, @@ -1784,7 +1793,7 @@ mod tests { }, ); source.assigned_shards.insert( - 2, + ShardId::from(2), AssignedShard { leader_id: "test-ingester-0".into(), follower_id_opt: Some("test-ingester-1".into()), @@ -1794,7 +1803,7 @@ mod tests { }, ); source.assigned_shards.insert( - 3, + ShardId::from(3), AssignedShard { leader_id: "test-ingester-1".into(), follower_id_opt: Some("test-ingester-0".into()), @@ -1804,7 +1813,7 @@ mod tests { }, ); source.assigned_shards.insert( - 4, + ShardId::from(4), AssignedShard { leader_id: "test-ingester-2".into(), follower_id_opt: Some("test-ingester-3".into()), @@ -1814,7 +1823,7 @@ mod tests { }, ); source.assigned_shards.insert( - 5, + ShardId::from(5), AssignedShard { leader_id: "test-ingester-2".into(), follower_id_opt: Some("test-ingester-3".into()), @@ -1841,12 +1850,12 @@ mod tests { source_id: "test-source".to_string(), }, vec![ - (1u64, Position::offset(11u64)), - (2u64, Position::offset(22u64)), - (3u64, Position::eof(33u64)), - (4u64, Position::offset(44u64)), - (5u64, Position::Beginning), - (6u64, Position::offset(66u64)), + (ShardId::from(1u64), Position::offset(11u64)), + (ShardId::from(2u64), Position::offset(22u64)), + (ShardId::from(3u64), Position::eof(33u64)), + (ShardId::from(4u64), Position::offset(44u64)), + (ShardId::from(5u64), Position::Beginning), + (ShardId::from(6u64), Position::offset(66u64)), ], ); assert_eq!(local_shards_update, expected_local_shards_update); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs index 2004169d347..a4ff25f6264 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs @@ -47,7 +47,7 @@ const BROADCAST_INTERVAL_PERIOD: Duration = if cfg!(test) { const ONE_MIB: ByteSize = ByteSize::mib(1); /// Broadcasted information about a primary shard. -#[derive(Debug, Clone, Copy, Eq, PartialEq, Ord, PartialOrd)] +#[derive(Debug, Clone, Eq, PartialEq, Ord, PartialOrd)] pub struct ShardInfo { pub shard_id: ShardId, pub shard_state: ShardState, @@ -72,11 +72,10 @@ impl<'de> Deserialize<'de> for ShardInfo { let value = String::deserialize(deserializer)?; let mut parts = value.split(':'); - let shard_id = parts + let shard_id: ShardId = parts .next() .ok_or_else(|| serde::de::Error::custom("invalid shard info"))? - .parse::() - .map_err(|_| serde::de::Error::custom("invalid shard ID"))?; + .into(); let shard_state_str = parts .next() @@ -351,12 +350,12 @@ mod tests { #[test] fn test_shard_info_serde() { let shard_info = ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(42), }; let serialized = serde_json::to_string(&shard_info).unwrap(); - assert_eq!(serialized, r#""1:open:42""#); + assert_eq!(serialized, r#""00000000000000000001:open:42""#); let deserialized = serde_json::from_str::(&serialized).unwrap(); assert_eq!(deserialized, shard_info); @@ -377,7 +376,7 @@ mod tests { source_id: "test-source".to_string(), }, vec![ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(42), }] @@ -417,7 +416,7 @@ mod tests { source_id: "test-source".to_string(), }, vec![ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, ingestion_rate: RateMibPerSec(42), }] @@ -493,7 +492,7 @@ mod tests { let mut state_guard = state.write().await; - let queue_id_01 = queue_id("test-index:0", "test-source", 1); + let queue_id_01 = queue_id("test-index:0", "test-source", &ShardId::from(1)); let shard = IngesterShard::new_solo(ShardState::Open, Position::Beginning, Position::Beginning); state_guard.shards.insert(queue_id_01.clone(), shard); @@ -568,7 +567,7 @@ mod tests { assert_eq!(event.shard_infos.len(), 1); let shard_info = event.shard_infos.iter().next().unwrap(); - assert_eq!(shard_info.shard_id, 1); + assert_eq!(shard_info.shard_id, ShardId::from(1)); assert_eq!(shard_info.shard_state, ShardState::Open); assert_eq!(shard_info.ingestion_rate, 42u16); }) @@ -584,7 +583,7 @@ mod tests { }; let key = make_key(&source_uid); let value = serde_json::to_string(&vec![ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(42), }]) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index c3ad918c394..364dae139a2 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -86,11 +86,11 @@ impl FetchStreamTask { .unwrap_or_default(); let (fetch_message_tx, fetch_stream) = ServiceStream::new_bounded(3); let mut fetch_task = Self { + shard_id: open_fetch_stream_request.shard_id().clone(), queue_id: open_fetch_stream_request.queue_id(), client_id: open_fetch_stream_request.client_id, index_uid: open_fetch_stream_request.index_uid.into(), source_id: open_fetch_stream_request.source_id, - shard_id: open_fetch_stream_request.shard_id, from_position_inclusive, state, fetch_message_tx, @@ -163,7 +163,7 @@ impl FetchStreamTask { let fetch_payload = FetchPayload { index_uid: self.index_uid.clone().into(), source_id: self.source_id.clone(), - shard_id: self.shard_id, + shard_id: Some(self.shard_id.clone()), mrecord_batch: Some(mrecord_batch), from_position_exclusive: Some(from_position_exclusive), to_position_inclusive: Some(to_position_inclusive.clone()), @@ -196,7 +196,7 @@ impl FetchStreamTask { let fetch_eof = FetchEof { index_uid: self.index_uid.clone().into(), source_id: self.source_id.clone(), - shard_id: self.shard_id, + shard_id: Some(self.shard_id.clone()), eof_position: Some(eof_position), }; let fetch_message = FetchMessage::new_eof(fetch_eof); @@ -280,7 +280,7 @@ impl MultiFetchStream { shard_id: ShardId, from_position_exclusive: Position, ) -> IngestV2Result<()> { - let queue_id = queue_id(index_uid.as_str(), &source_id, shard_id); + let queue_id = queue_id(index_uid.as_str(), &source_id, &shard_id); let entry = self.fetch_task_handles.entry(queue_id.clone()); if let Entry::Occupied(_) = entry { @@ -317,9 +317,9 @@ impl MultiFetchStream { &mut self, index_uid: &str, source_id: &str, - shard_id: u64, + shard_id: ShardId, ) -> IngestV2Result<()> { - let queue_id = queue_id(index_uid, source_id, shard_id); + let queue_id = queue_id(index_uid, source_id, &shard_id); if let Some(fetch_stream_handle) = self.fetch_task_handles.remove(&queue_id) { fetch_stream_handle.abort(); @@ -338,7 +338,7 @@ impl MultiFetchStream { self.fetch_message_rx .recv() .await - .expect("the channel should be open") + .expect("channel should be open") } /// Resets the stream by aborting all the active fetch tasks and dropping all queued responses. @@ -401,7 +401,7 @@ async fn retrying_fetch_stream( client_id.clone(), index_uid.clone(), source_id.clone(), - shard_id, + shard_id.clone(), &mut from_position_exclusive, &ingester_ids, ingester_pool.clone(), @@ -471,7 +471,7 @@ async fn fault_tolerant_fetch_stream( client_id: client_id.clone(), index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id, + shard_id: Some(shard_id.clone()), from_position_exclusive: Some(from_position_exclusive.clone()), }; let mut fetch_stream = match ingester.open_fetch_stream(open_fetch_stream_request).await { @@ -529,7 +529,7 @@ async fn fault_tolerant_fetch_stream( match fetch_message_result { Ok(fetch_message) => match &fetch_message.message { Some(fetch_message::Message::Payload(fetch_payload)) => { - let to_position_inclusive = fetch_payload.to_position_inclusive(); + let to_position_inclusive = fetch_payload.to_position_inclusive().clone(); if fetch_message_tx.send(Ok(fetch_message)).await.is_err() { // The consumer was dropped. @@ -538,7 +538,7 @@ async fn fault_tolerant_fetch_stream( *from_position_exclusive = to_position_inclusive; } Some(fetch_message::Message::Eof(fetch_eof)) => { - let eof_position = fetch_eof.eof_position(); + let eof_position = fetch_eof.eof_position().clone(); // We ignore the send error if the consumer was dropped because we're going // to return anyway. @@ -628,8 +628,8 @@ pub(super) mod tests { client_id: client_id.clone(), index_uid: index_uid.clone(), source_id: source_id.clone(), - shard_id: 1, - from_position_exclusive: None, + shard_id: Some(ShardId::from(1)), + from_position_exclusive: Some(Position::Beginning), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); let state = Arc::new(RwLock::new(IngesterState { @@ -648,7 +648,7 @@ pub(super) mod tests { shard_status_rx, 1024, ); - let queue_id = queue_id(&index_uid, &source_id, 1); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let mut state_guard = state.write().await; @@ -673,7 +673,7 @@ pub(super) mod tests { assert_eq!(fetch_payload.index_uid, "test-index:0"); assert_eq!(fetch_payload.source_id, "test-source"); - assert_eq!(fetch_payload.shard_id, 1); + assert_eq!(fetch_payload.shard_id(), ShardId::from(1)); assert_eq!(fetch_payload.from_position_exclusive(), Position::Beginning); assert_eq!( fetch_payload.to_position_inclusive(), @@ -802,7 +802,7 @@ pub(super) mod tests { assert_eq!(fetch_eof.index_uid, "test-index:0"); assert_eq!(fetch_eof.source_id, "test-source"); - assert_eq!(fetch_eof.shard_id, 1); + assert_eq!(fetch_eof.shard_id(), ShardId::from(1)); assert_eq!(fetch_eof.eof_position, Some(Position::eof(3u64))); fetch_task_handle.await.unwrap(); @@ -819,8 +819,8 @@ pub(super) mod tests { client_id: client_id.clone(), index_uid: index_uid.clone(), source_id: source_id.clone(), - shard_id: 1, - from_position_exclusive: None, + shard_id: Some(ShardId::from(1)), + from_position_exclusive: Some(Position::Beginning), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); let state = Arc::new(RwLock::new(IngesterState { @@ -839,7 +839,7 @@ pub(super) mod tests { shard_status_rx, 1024, ); - let queue_id = queue_id(&index_uid, &source_id, 1); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let mut state_guard = state.write().await; @@ -866,7 +866,7 @@ pub(super) mod tests { assert_eq!(fetch_eof.index_uid, "test-index:0"); assert_eq!(fetch_eof.source_id, "test-source"); - assert_eq!(fetch_eof.shard_id, 1); + assert_eq!(fetch_eof.shard_id(), ShardId::from(1)); assert_eq!(fetch_eof.eof_position, Some(Position::Beginning.as_eof())); fetch_task_handle.await.unwrap(); @@ -883,7 +883,7 @@ pub(super) mod tests { client_id: client_id.clone(), index_uid: index_uid.clone(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), from_position_exclusive: Some(Position::offset(0u64)), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); @@ -903,7 +903,7 @@ pub(super) mod tests { shard_status_rx, 1024, ); - let queue_id = queue_id(&index_uid, &source_id, 1); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let mut state_guard = state.write().await; @@ -955,7 +955,7 @@ pub(super) mod tests { assert_eq!(fetch_payload.index_uid, "test-index:0"); assert_eq!(fetch_payload.source_id, "test-source"); - assert_eq!(fetch_payload.shard_id, 1); + assert_eq!(fetch_payload.shard_id(), ShardId::from(1)); assert_eq!( fetch_payload.from_position_exclusive(), Position::offset(0u64) @@ -989,8 +989,8 @@ pub(super) mod tests { client_id: client_id.clone(), index_uid: index_uid.clone(), source_id: source_id.clone(), - shard_id: 1, - from_position_exclusive: None, + shard_id: Some(ShardId::from(1)), + from_position_exclusive: Some(Position::Beginning), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); let state = Arc::new(RwLock::new(IngesterState { @@ -1030,8 +1030,8 @@ pub(super) mod tests { client_id: client_id.clone(), index_uid: index_uid.clone(), source_id: source_id.clone(), - shard_id: 1, - from_position_exclusive: None, + shard_id: Some(ShardId::from(1)), + from_position_exclusive: Some(Position::Beginning), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); let state = Arc::new(RwLock::new(IngesterState { @@ -1050,7 +1050,7 @@ pub(super) mod tests { shard_status_rx, 30, ); - let queue_id = queue_id(&index_uid, &source_id, 1); + let queue_id = queue_id(&index_uid, &source_id, &ShardId::from(1)); let mut state_guard = state.write().await; @@ -1149,7 +1149,7 @@ pub(super) mod tests { let client_id = "test-client".to_string(); let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); - let shard_id: ShardId = 1; + let shard_id = ShardId::from(1); let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; @@ -1165,7 +1165,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) @@ -1177,7 +1177,7 @@ pub(super) mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(1u64)), @@ -1188,7 +1188,7 @@ pub(super) mod tests { let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), eof_position: Some(Position::eof(1u64)), }; let fetch_message = FetchMessage::new_eof(fetch_eof); @@ -1242,7 +1242,7 @@ pub(super) mod tests { let client_id = "test-client".to_string(); let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); - let shard_id: ShardId = 1; + let shard_id = ShardId::from(1); let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; @@ -1258,7 +1258,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Err(IngestV2Error::Internal( @@ -1274,7 +1274,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) @@ -1287,7 +1287,7 @@ pub(super) mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(1u64)), @@ -1298,7 +1298,7 @@ pub(super) mod tests { let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), eof_position: Some(Position::eof(1u64)), }; let fetch_message = FetchMessage::new_eof(fetch_eof); @@ -1352,7 +1352,7 @@ pub(super) mod tests { let client_id = "test-client".to_string(); let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); - let shard_id: ShardId = 1; + let shard_id = ShardId::from(1); let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; @@ -1369,7 +1369,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_0) @@ -1383,7 +1383,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(1u64)); Ok(service_stream_1) @@ -1396,7 +1396,7 @@ pub(super) mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(1u64)), @@ -1410,7 +1410,7 @@ pub(super) mod tests { let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), eof_position: Some(Position::eof(1u64)), }; let fetch_message = FetchMessage::new_eof(fetch_eof); @@ -1464,7 +1464,7 @@ pub(super) mod tests { let client_id = "test-client".to_string(); let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); - let shard_id: ShardId = 1; + let shard_id = ShardId::from(1); let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; @@ -1479,10 +1479,12 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); - Err(IngestV2Error::ShardNotFound { shard_id: 1 }) + Err(IngestV2Error::ShardNotFound { + shard_id: ShardId::from(1), + }) }); let ingester_0: IngesterServiceClient = ingester_mock_0.into(); ingester_pool.insert("test-ingester-0".into(), ingester_0); @@ -1507,7 +1509,7 @@ pub(super) mod tests { assert!(matches!( fetch_stream_error.ingest_error, - IngestV2Error::ShardNotFound { shard_id: 1 } + IngestV2Error::ShardNotFound { shard_id } if shard_id == ShardId::from(1) )); assert!(from_position_exclusive.is_eof()); } @@ -1517,7 +1519,7 @@ pub(super) mod tests { let client_id = "test-client".to_string(); let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); - let shard_id: ShardId = 1; + let shard_id = ShardId::from(1); let from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester".into()]; @@ -1538,7 +1540,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Err(IngestV2Error::Internal( @@ -1552,7 +1554,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) @@ -1564,7 +1566,7 @@ pub(super) mod tests { assert_eq!(request.client_id, "test-client"); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); - assert_eq!(request.shard_id, 1); + assert_eq!(request.shard_id(), ShardId::from(1)); assert_eq!(request.from_position_exclusive(), Position::offset(1u64)); Ok(service_stream_2) @@ -1576,7 +1578,7 @@ pub(super) mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(1u64)), @@ -1590,7 +1592,7 @@ pub(super) mod tests { let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-bar"]), from_position_exclusive: Some(Position::offset(1u64)), to_position_inclusive: Some(Position::offset(2u64)), diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index b9342050706..aa422e3f13e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -776,7 +776,7 @@ impl Ingester { .shards .get(&queue_id) .ok_or(IngestV2Error::ShardNotFound { - shard_id: open_fetch_stream_request.shard_id, + shard_id: open_fetch_stream_request.shard_id().clone(), })? .shard_status_rx .clone(); @@ -998,7 +998,7 @@ impl IngesterService for Ingester { queue_id( &retain_shards_for_source.index_uid, &retain_shards_for_source.source_id, - shard_id, + &shard_id, ) }) }) @@ -1070,7 +1070,7 @@ impl IngesterState { async fn truncate_shard( &mut self, queue_id: &QueueId, - truncate_up_to_position_inclusive: Position, + truncate_up_to_position_inclusive: &Position, ) { // TODO: Replace with if-let-chains when stabilized. let Some(truncate_up_to_offset_inclusive) = truncate_up_to_position_inclusive.as_u64() @@ -1080,7 +1080,7 @@ impl IngesterState { let Some(shard) = self.shards.get_mut(queue_id) else { return; }; - if shard.truncation_position_inclusive >= truncate_up_to_position_inclusive { + if shard.truncation_position_inclusive >= *truncate_up_to_position_inclusive { return; } match self @@ -1089,7 +1089,7 @@ impl IngesterState { .await { Ok(_) => { - shard.truncation_position_inclusive = truncate_up_to_position_inclusive; + shard.truncation_position_inclusive = truncate_up_to_position_inclusive.clone(); } Err(TruncateError::MissingQueue(_)) => { error!("failed to truncate shard `{queue_id}`: WAL queue not found"); @@ -1133,12 +1133,12 @@ impl EventSubscriber for WeakIngesterState { let source_id = shard_positions_update.source_uid.source_id; for (shard_id, shard_position) in shard_positions_update.shard_positions { - let queue_id = queue_id(index_uid.as_str(), &source_id, shard_id); + let queue_id = queue_id(index_uid.as_str(), &source_id, &shard_id); if shard_position.is_eof() { state_guard.delete_shard(&queue_id).await; } else { - state_guard.truncate_shard(&queue_id, shard_position).await; + state_guard.truncate_shard(&queue_id, &shard_position).await; } } } @@ -1178,6 +1178,8 @@ pub async fn wait_for_ingester_decommission(ingester_opt: Option>(); @@ -900,25 +902,25 @@ mod tests { subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(0u64)), }, ReplicateSubrequest { subrequest_id: 1, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(1u64)), }, ReplicateSubrequest { subrequest_id: 2, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-qux", "test-doc-tux"])), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(2u64)), @@ -942,7 +944,7 @@ mod tests { let replicate_success_0 = &replicate_response.successes[0]; assert_eq!(replicate_success_0.index_uid, "test-index:0"); assert_eq!(replicate_success_0.source_id, "test-source"); - assert_eq!(replicate_success_0.shard_id, 1); + assert_eq!(replicate_success_0.shard_id(), ShardId::from(1)); assert_eq!( replicate_success_0.replication_position_inclusive(), Position::offset(0u64) @@ -951,7 +953,7 @@ mod tests { let replicate_success_1 = &replicate_response.successes[1]; assert_eq!(replicate_success_1.index_uid, "test-index:0"); assert_eq!(replicate_success_1.source_id, "test-source"); - assert_eq!(replicate_success_1.shard_id, 2); + assert_eq!(replicate_success_1.shard_id(), ShardId::from(2)); assert_eq!( replicate_success_1.replication_position_inclusive(), Position::offset(1u64) @@ -960,7 +962,7 @@ mod tests { let replicate_success_2 = &replicate_response.successes[2]; assert_eq!(replicate_success_2.index_uid, "test-index:1"); assert_eq!(replicate_success_2.source_id, "test-source"); - assert_eq!(replicate_success_2.shard_id, 1); + assert_eq!(replicate_success_2.shard_id(), ShardId::from(1)); assert_eq!( replicate_success_2.replication_position_inclusive(), Position::offset(2u64) @@ -1044,7 +1046,7 @@ mod tests { replica_shard: Some(Shard { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-leader".to_string(), follower_id: Some("test-follower".to_string()), @@ -1067,7 +1069,7 @@ mod tests { replica_shard: Some(Shard { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, leader_id: "test-leader".to_string(), follower_id: Some("test-follower".to_string()), @@ -1090,7 +1092,7 @@ mod tests { replica_shard: Some(Shard { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-leader".to_string(), follower_id: Some("test-follower".to_string()), @@ -1110,7 +1112,7 @@ mod tests { let state_guard = state.read().await; - let queue_id_01 = queue_id("test-index:0", "test-source", 1); + let queue_id_01 = queue_id("test-index:0", "test-source", &ShardId::from(1)); let replica_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); replica_shard_01.assert_is_replica(); @@ -1120,7 +1122,7 @@ mod tests { assert!(state_guard.mrecordlog.queue_exists(&queue_id_01)); - let queue_id_02 = queue_id("test-index:0", "test-source", 2); + let queue_id_02 = queue_id("test-index:0", "test-source", &ShardId::from(2)); let replica_shard_02 = state_guard.shards.get(&queue_id_02).unwrap(); replica_shard_02.assert_is_replica(); @@ -1128,7 +1130,7 @@ mod tests { replica_shard_02.assert_replication_position(Position::Beginning); replica_shard_02.assert_truncation_position(Position::Beginning); - let queue_id_11 = queue_id("test-index:1", "test-source", 1); + let queue_id_11 = queue_id("test-index:1", "test-source", &ShardId::from(1)); let replica_shard_11 = state_guard.shards.get(&queue_id_11).unwrap(); replica_shard_11.assert_is_replica(); @@ -1147,27 +1149,27 @@ mod tests { subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(0u64)), }, ReplicateSubrequest { subrequest_id: 1, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(1u64)), }, ReplicateSubrequest { subrequest_id: 2, index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-qux", "test-doc-tux"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(1u64)), }, ], @@ -1190,7 +1192,7 @@ mod tests { let replicate_success_0 = &replicate_response.successes[0]; assert_eq!(replicate_success_0.index_uid, "test-index:0"); assert_eq!(replicate_success_0.source_id, "test-source"); - assert_eq!(replicate_success_0.shard_id, 1); + assert_eq!(replicate_success_0.shard_id(), ShardId::from(1)); assert_eq!( replicate_success_0.replication_position_inclusive(), Position::offset(0u64) @@ -1199,7 +1201,7 @@ mod tests { let replicate_success_1 = &replicate_response.successes[1]; assert_eq!(replicate_success_1.index_uid, "test-index:0"); assert_eq!(replicate_success_1.source_id, "test-source"); - assert_eq!(replicate_success_1.shard_id, 2); + assert_eq!(replicate_success_1.shard_id(), ShardId::from(2)); assert_eq!( replicate_success_1.replication_position_inclusive(), Position::offset(1u64) @@ -1208,7 +1210,7 @@ mod tests { let replicate_success_2 = &replicate_response.successes[2]; assert_eq!(replicate_success_2.index_uid, "test-index:1"); assert_eq!(replicate_success_2.source_id, "test-source"); - assert_eq!(replicate_success_2.shard_id, 1); + assert_eq!(replicate_success_2.shard_id(), ShardId::from(1)); assert_eq!( replicate_success_2.replication_position_inclusive(), Position::offset(1u64) @@ -1241,7 +1243,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-moo"])), from_position_exclusive: Some(Position::offset(0u64)), to_position_inclusive: Some(Position::offset(1u64)), @@ -1265,7 +1267,7 @@ mod tests { let replicate_success_0 = &replicate_response.successes[0]; assert_eq!(replicate_success_0.index_uid, "test-index:0"); assert_eq!(replicate_success_0.source_id, "test-source"); - assert_eq!(replicate_success_0.shard_id, 1); + assert_eq!(replicate_success_0.shard_id(), ShardId::from(1)); assert_eq!( replicate_success_0.replication_position_inclusive(), Position::offset(1u64) @@ -1314,7 +1316,7 @@ mod tests { memory_capacity, ); - let queue_id_01 = queue_id("test-index:0", "test-source", 1); + let queue_id_01 = queue_id("test-index:0", "test-source", &ShardId::from(1)); let replica_shard = IngesterShard::new_replica( leader_id, ShardState::Closed, @@ -1335,7 +1337,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: Position::offset(0u64).into(), to_position_inclusive: Some(Position::offset(1u64)), @@ -1358,7 +1360,7 @@ mod tests { let replicate_failure = &replicate_response.failures[0]; assert_eq!(replicate_failure.index_uid, "test-index:0"); assert_eq!(replicate_failure.source_id, "test-source"); - assert_eq!(replicate_failure.shard_id, 1); + assert_eq!(replicate_failure.shard_id(), ShardId::from(1)); assert_eq!( replicate_failure.reason(), ReplicateFailureReason::ShardClosed @@ -1399,7 +1401,7 @@ mod tests { memory_capacity, ); - let queue_id_01 = queue_id("test-index:0", "test-source", 1); + let queue_id_01 = queue_id("test-index:0", "test-source", &ShardId::from(1)); let replica_shard = IngesterShard::new_replica( leader_id, ShardState::Open, @@ -1420,9 +1422,9 @@ mod tests { subrequest_id: 0, index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - from_position_exclusive: None, + from_position_exclusive: Some(Position::Beginning), to_position_inclusive: Some(Position::offset(0u64)), }], replication_seqno: 0, @@ -1443,7 +1445,7 @@ mod tests { let replicate_failure_0 = &replicate_response.failures[0]; assert_eq!(replicate_failure_0.index_uid, "test-index:0"); assert_eq!(replicate_failure_0.source_id, "test-source"); - assert_eq!(replicate_failure_0.shard_id, 1); + assert_eq!(replicate_failure_0.shard_id(), ShardId::from(1)); assert_eq!( replicate_failure_0.reason(), ReplicateFailureReason::ResourceExhausted diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 868478cef32..c7d5f647ddc 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -230,20 +230,23 @@ impl IngestRouter { } for persist_failure in persist_response.failures { workbench.record_persist_failure(&persist_failure); + if persist_failure.reason() == PersistFailureReason::ShardClosed { + let shard_id = persist_failure.shard_id().clone(); let index_uid: IndexUid = persist_failure.index_uid.into(); let source_id: SourceId = persist_failure.source_id; closed_shards .entry((index_uid, source_id)) .or_default() - .push(persist_failure.shard_id); + .push(shard_id); } else if persist_failure.reason() == PersistFailureReason::ShardNotFound { + let shard_id = persist_failure.shard_id().clone(); let index_uid: IndexUid = persist_failure.index_uid.into(); let source_id: SourceId = persist_failure.source_id; deleted_shards .entry((index_uid, source_id)) .or_default() - .push(persist_failure.shard_id); + .push(shard_id); } } } @@ -333,7 +336,7 @@ impl IngestRouter { subrequest_id: subrequest.subrequest_id, index_uid: shard.index_uid.clone().into(), source_id: shard.source_id.clone(), - shard_id: shard.shard_id, + shard_id: Some(shard.shard_id.clone()), doc_batch: subrequest.doc_batch.clone(), }; per_leader_persist_subrequests @@ -558,14 +561,14 @@ mod tests { RoutingEntry { index_uid: "test-index-0:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index-0:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, @@ -612,7 +615,7 @@ mod tests { ShardIds { index_uid: "test-index-0:0".into(), source_id: "test-source".to_string(), - shard_ids: vec![1], + shard_ids: vec![ShardId::from(1)], } ); assert_eq!( @@ -703,7 +706,7 @@ mod tests { open_shards: vec![Shard { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, ..Default::default() }], @@ -716,14 +719,14 @@ mod tests { Shard { index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, ..Default::default() }, Shard { index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, ..Default::default() }, @@ -834,14 +837,14 @@ mod tests { .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, 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, 1); - assert_eq!(routing_entry_1.all_shards()[1].shard_id, 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!( @@ -888,7 +891,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), ..Default::default() }], failures: Vec::new(), @@ -939,7 +942,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), reason: PersistFailureReason::RateLimited as i32, }], }); @@ -975,14 +978,14 @@ mod tests { vec![ Shard { index_uid: "test-index-0:0".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() }, Shard { index_uid: "test-index-0:0".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() @@ -1007,14 +1010,14 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), reason: PersistFailureReason::ShardNotFound as i32, }, PersistFailure { subrequest_id: 1, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), reason: PersistFailureReason::ShardClosed as i32, }, ], @@ -1033,7 +1036,7 @@ mod tests { assert_eq!(routing_table_entry.len(), 1); let shard = routing_table_entry.all_shards()[0]; - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id, ShardId::from(2)); assert_eq!(shard.shard_state, ShardState::Closed); } @@ -1144,7 +1147,7 @@ mod tests { vec![Shard { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() @@ -1157,7 +1160,7 @@ mod tests { Shard { index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + 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()), @@ -1166,7 +1169,7 @@ mod tests { Shard { index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-1".to_string(), follower_id: Some("test-ingester-2".to_string()), @@ -1189,7 +1192,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-foo", "test-doc-bar"])) @@ -1199,7 +1202,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 1); assert_eq!(subrequest.index_uid, "test-index-1:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-qux"])) @@ -1212,14 +1215,14 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(1u64)), }, PersistSuccess { subrequest_id: 1, index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), }, ], @@ -1239,7 +1242,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-moo", "test-doc-baz"])) @@ -1251,7 +1254,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(3u64)), }], failures: Vec::new(), @@ -1274,7 +1277,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 1); assert_eq!(subrequest.index_uid, "test-index-1:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 2); + assert_eq!(subrequest.shard_id(), ShardId::from(2)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-tux"])) @@ -1286,7 +1289,7 @@ mod tests { subrequest_id: 1, index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), replication_position_inclusive: Some(Position::offset(0u64)), }], failures: Vec::new(), @@ -1354,7 +1357,7 @@ mod tests { vec![Shard { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester-0".to_string(), ..Default::default() @@ -1375,7 +1378,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-foo"])) @@ -1388,7 +1391,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), reason: PersistFailureReason::RateLimited as i32, }], }; @@ -1406,7 +1409,7 @@ mod tests { assert_eq!(subrequest.subrequest_id, 0); assert_eq!(subrequest.index_uid, "test-index-0:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.shard_id(), ShardId::from(1)); assert_eq!( subrequest.doc_batch, Some(DocBatchV2::for_test(["test-doc-foo"])) @@ -1418,7 +1421,7 @@ mod tests { subrequest_id: 0, index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), }], failures: Vec::new(), @@ -1461,7 +1464,7 @@ mod tests { "test-source", vec![Shard { index_uid: "test-index-0:0".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-ingester".to_string(), ..Default::default() @@ -1477,12 +1480,12 @@ mod tests { }, shard_infos: BTreeSet::from_iter([ ShardInfo { - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, ingestion_rate: RateMibPerSec(0), }, ShardInfo { - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, ingestion_rate: RateMibPerSec(0), }, @@ -1500,9 +1503,9 @@ mod tests { .unwrap() .all_shards(); assert_eq!(shards.len(), 2); - assert_eq!(shards[0].shard_id, 1); + assert_eq!(shards[0].shard_id, ShardId::from(1)); assert_eq!(shards[0].shard_state, ShardState::Closed); - assert_eq!(shards[1].shard_id, 2); + assert_eq!(shards[1].shard_id, ShardId::from(2)); assert_eq!(shards[1].shard_state, ShardState::Open); drop(state_guard); @@ -1511,7 +1514,7 @@ mod tests { index_uid: "test-index-0:0".into(), source_id: "test-source".to_string(), }, - shard_positions: vec![(1, Position::eof(0u64))], + shard_positions: vec![(ShardId::from(1), Position::eof(0u64))], }; event_broker.publish(shard_positions_update); @@ -1525,7 +1528,7 @@ mod tests { .unwrap() .all_shards(); assert_eq!(shards.len(), 1); - assert_eq!(shards[0].shard_id, 2); + assert_eq!(shards[0].shard_id, ShardId::from(2)); drop(state_guard); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs index 2e598728f41..1aeaf31372b 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/routing_table.rs @@ -38,11 +38,12 @@ pub(super) struct RoutingEntry { 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.into(), source_id: shard.source_id, - shard_id: shard.shard_id, + shard_id, shard_state, leader_id: shard.leader_id.into(), } @@ -71,8 +72,8 @@ impl RoutingTableEntry { ) -> Self { let num_shards = shards.len(); - shards.sort_unstable_by_key(|shard| shard.shard_id); - shards.dedup_by_key(|shard| shard.shard_id); + shards.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); + shards.dedup_by(|left, right| left.shard_id == right.shard_id); let (local_shards, remote_shards): (Vec<_>, Vec<_>) = shards .into_iter() @@ -88,8 +89,10 @@ impl RoutingTableEntry { index_uid, source_id, local_shards, + // local_shard_ids_range_opt, local_round_robin_idx: AtomicUsize::default(), remote_shards, + // remote_shard_ids_range_opt, remote_round_robin_idx: AtomicUsize::default(), } } @@ -120,7 +123,7 @@ impl RoutingTableEntry { for shard in shards { match shard.shard_state { ShardState::Closed => { - closed_shard_ids.push(shard.shard_id); + closed_shard_ids.push(shard.shard_id.clone()); continue; } ShardState::Unavailable | ShardState::Unspecified => { @@ -197,31 +200,31 @@ impl RoutingTableEntry { if num_target_shards == 0 { target_shards.reserve(num_target_shards); - target_shards.extend(shard_ids.iter().map(|&shard_id| RoutingEntry { + 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: 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..=target_shards[num_target_shards - 1].shard_id; + 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 { + for shard_id in shard_ids { // If we can't find the shard, then we insert it. - if shard_ids_range.contains(&shard_id) { + if shard_ids_range.contains(shard_id) { continue; } if target_shards[..num_target_shards] - .binary_search_by_key(&shard_id, |shard| shard.shard_id) + .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: shard_id.clone(), shard_state: ShardState::Open, leader_id: leader_id.clone(), }); @@ -230,7 +233,7 @@ impl RoutingTableEntry { } } if num_inserted_shards > 0 { - target_shards.sort_unstable_by_key(|shard| shard.shard_id); + target_shards.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); info!( index_id=%self.index_uid.index_id(), @@ -260,13 +263,14 @@ impl RoutingTableEntry { continue; } let num_shards = shards.len(); - let shard_ids_range = shards[0].shard_id..=shards[num_shards - 1].shard_id; + 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_key(shard_id, |shard| shard.shard_id) + if let Ok(shard_idx) = shards.binary_search_by(|shard| shard.shard_id.cmp(shard_id)) { shards[shard_idx].shard_state = ShardState::Closed; } @@ -286,14 +290,15 @@ impl RoutingTableEntry { continue; } let num_shards = shards.len(); - let shard_ids_range = shards[0].shard_id..=shards[num_shards - 1].shard_id; + 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_key(shard_id, |shard| shard.shard_id) + 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; @@ -487,7 +492,7 @@ mod tests { Shard { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: Some(ShardId::from(3)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), ..Default::default() @@ -495,7 +500,7 @@ mod tests { Shard { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), ..Default::default() @@ -503,7 +508,7 @@ mod tests { Shard { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: Some(ShardId::from(2)), shard_state: ShardState::Open as i32, leader_id: "test-node-1".to_string(), ..Default::default() @@ -511,7 +516,7 @@ mod tests { Shard { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Open as i32, leader_id: "test-node-0".to_string(), ..Default::default() @@ -519,7 +524,7 @@ mod tests { Shard { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 4, + shard_id: Some(ShardId::from(4)), shard_state: ShardState::Closed as i32, leader_id: "test-node-0".to_string(), ..Default::default() @@ -527,11 +532,11 @@ mod tests { ]; 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, 1); - assert_eq!(table_entry.local_shards[1].shard_id, 3); + 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, 2); + assert_eq!(table_entry.remote_shards[0].shard_id, ShardId::from(2)); } #[test] @@ -568,14 +573,14 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, @@ -590,7 +595,7 @@ mod tests { &mut unavailable_leaders )); assert_eq!(closed_shard_ids.len(), 1); - assert_eq!(closed_shard_ids[0], 1); + assert_eq!(closed_shard_ids[0], ShardId::from(1)); assert!(unavailable_leaders.is_empty()); closed_shard_ids.clear(); @@ -604,21 +609,21 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-2".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, @@ -631,7 +636,7 @@ mod tests { &mut unavailable_leaders )); assert_eq!(closed_shard_ids.len(), 1); - assert_eq!(closed_shard_ids[0], 1); + assert_eq!(closed_shard_ids[0], ShardId::from(1)); assert_eq!(unavailable_leaders.len(), 1); assert!(unavailable_leaders.contains("test-ingester-2")); } @@ -662,21 +667,21 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, @@ -688,17 +693,17 @@ mod tests { let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id, ShardId::from(2)); let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 3); + assert_eq!(shard.shard_id, ShardId::from(3)); let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id, ShardId::from(2)); let table_entry = RoutingTableEntry { index_uid: index_uid.clone(), @@ -706,7 +711,7 @@ mod tests { local_shards: vec![RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Closed, leader_id: "test-ingester-0".into(), }], @@ -715,28 +720,28 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Closed, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 4, + shard_id: ShardId::from(4), shard_state: ShardState::Open, leader_id: "test-ingester-2".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 5, + shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, @@ -746,17 +751,17 @@ mod tests { let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id, ShardId::from(2)); let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 5); + assert_eq!(shard.shard_id, ShardId::from(5)); let shard = table_entry .next_open_shard_round_robin(&ingester_pool) .unwrap(); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id, ShardId::from(2)); } #[test] @@ -772,54 +777,79 @@ mod tests { 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, &[2]); + table_entry.insert_open_shards( + &local_node_id, + &local_node_id, + &index_uid_0, + &[ShardId::from(2)], + ); 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, 2); + 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); table_entry.local_shards[0].shard_state = ShardState::Closed; - table_entry.insert_open_shards(&local_node_id, &local_node_id, &index_uid_0, &[1, 2]); + table_entry.insert_open_shards( + &local_node_id, + &local_node_id, + &index_uid_0, + &[ShardId::from(1), ShardId::from(2)], + ); 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, 1); + 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, 2); + 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, &[2]); + table_entry.insert_open_shards( + &local_node_id, + &remote_node_id, + &index_uid_0, + &[ShardId::from(2)], + ); assert_eq!(table_entry.local_shards.len(), 0); assert_eq!(table_entry.remote_shards.len(), 1); 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, 2); + 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); table_entry.remote_shards[0].shard_state = ShardState::Closed; - table_entry.insert_open_shards(&local_node_id, &remote_node_id, &index_uid_0, &[1, 2]); + table_entry.insert_open_shards( + &local_node_id, + &remote_node_id, + &index_uid_0, + &[ShardId::from(1), ShardId::from(2)], + ); assert_eq!(table_entry.local_shards.len(), 0); assert_eq!(table_entry.remote_shards.len(), 2); - assert_eq!(table_entry.remote_shards[0].shard_id, 1); + 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, 2); + assert_eq!(table_entry.remote_shards[1].shard_id, ShardId::from(2)); assert_eq!(table_entry.remote_shards[1].shard_state, ShardState::Closed); // Update index incarnation. let index_uid_1: IndexUid = IndexUid::new_2("test-index", 1); - table_entry.insert_open_shards(&local_node_id, &local_node_id, &index_uid_1, &[1]); + table_entry.insert_open_shards( + &local_node_id, + &local_node_id, + &index_uid_1, + &[ShardId::from(1)], + ); assert_eq!(table_entry.index_uid, index_uid_1); assert_eq!(table_entry.local_shards.len(), 1); @@ -827,7 +857,7 @@ mod tests { 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, 1); + 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); @@ -836,7 +866,7 @@ mod tests { &local_node_id, &local_node_id, &index_uid_0, - &[12, 42, 1337], + &[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); @@ -850,7 +880,7 @@ mod tests { let mut table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); table_entry.close_shards(&index_uid, &[]); - table_entry.close_shards(&index_uid, &[1]); + table_entry.close_shards(&index_uid, &[ShardId::from(1)]); assert!(table_entry.local_shards.is_empty()); assert!(table_entry.remote_shards.is_empty()); @@ -861,21 +891,21 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, @@ -885,28 +915,37 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 5, + shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 6, + shard_id: ShardId::from(6), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 7, + 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, &[1, 3, 4, 6, 8]); + table_entry.close_shards( + &index_uid, + &[ + ShardId::from(1), + ShardId::from(3), + ShardId::from(4), + ShardId::from(6), + ShardId::from(8), + ], + ); 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()); @@ -922,7 +961,7 @@ mod tests { let mut table_entry = RoutingTableEntry::empty(index_uid.clone(), source_id.clone()); table_entry.delete_shards(&index_uid, &[]); - table_entry.delete_shards(&index_uid, &[1]); + table_entry.delete_shards(&index_uid, &[ShardId::from(1)]); assert!(table_entry.local_shards.is_empty()); assert!(table_entry.remote_shards.is_empty()); @@ -933,21 +972,21 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: ShardId::from(1), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 2, + shard_id: ShardId::from(2), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 3, + shard_id: ShardId::from(3), shard_state: ShardState::Open, leader_id: "test-ingester-0".into(), }, @@ -957,32 +996,41 @@ mod tests { RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 5, + shard_id: ShardId::from(5), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 6, + shard_id: ShardId::from(6), shard_state: ShardState::Open, leader_id: "test-ingester-1".into(), }, RoutingEntry { index_uid: "test-index:0".into(), source_id: "test-source".to_string(), - shard_id: 7, + 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, &[1, 3, 4, 6, 8]); + table_entry.delete_shards( + &index_uid, + &[ + ShardId::from(1), + ShardId::from(3), + ShardId::from(4), + ShardId::from(6), + ShardId::from(8), + ], + ); assert_eq!(table_entry.local_shards.len(), 1); - assert_eq!(table_entry.local_shards[0].shard_id, 2); + 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, 5); - assert_eq!(table_entry.remote_shards[1].shard_id, 7); + 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/workbench.rs b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs index 28e305221a4..e7b1758c059 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/workbench.rs @@ -280,6 +280,7 @@ impl IngestSubworkbench { #[cfg(test)] mod tests { use quickwit_proto::ingest::ingester::PersistFailureReason; + use quickwit_proto::types::ShardId; use super::*; @@ -487,7 +488,7 @@ mod tests { let persist_failure = PersistFailure { subrequest_id: 0, - shard_id: 1, + shard_id: Some(ShardId::from(1)), reason: PersistFailureReason::ResourceExhausted as i32, ..Default::default() }; diff --git a/quickwit/quickwit-metastore/src/checkpoint.rs b/quickwit/quickwit-metastore/src/checkpoint.rs index 8e1ee18ee5f..159e0c7a39f 100644 --- a/quickwit/quickwit-metastore/src/checkpoint.rs +++ b/quickwit/quickwit-metastore/src/checkpoint.rs @@ -46,6 +46,10 @@ impl PartitionId { pub fn as_u64(&self) -> Option { self.0.parse().ok() } + + pub fn as_str(&self) -> &str { + &self.0 + } } impl fmt::Display for PartitionId { diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs index be6a97f5bf2..ab88c1785e3 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs @@ -79,9 +79,8 @@ pub(crate) struct FileBackedIndex { #[cfg(any(test, feature = "testsuite"))] impl quickwit_config::TestableForRegression for FileBackedIndex { fn sample_for_regression() -> Self { - use quickwit_proto::ingest::Shard; - - use self::shards::SerdeShards; + use quickwit_proto::ingest::{Shard, ShardState}; + use quickwit_proto::types::{Position, ShardId}; let index_metadata = IndexMetadata::sample_for_regression(); let index_uid = index_metadata.index_uid.clone(); @@ -99,17 +98,14 @@ impl quickwit_config::TestableForRegression for FileBackedIndex { let shard = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), + shard_state: ShardState::Open as i32, leader_id: "leader-ingester".to_string(), follower_id: Some("follower-ingester".to_string()), + publish_position_inclusive: Some(Position::Beginning), ..Default::default() }; - - let serde_shards = SerdeShards { - next_shard_id: 2, - shards: vec![shard], - }; - let shards = Shards::from_serde_shards(index_uid.clone(), source_id.clone(), serde_shards); + let shards = Shards::from_shards_vec(index_uid.clone(), source_id.clone(), vec![shard]); let per_source_shards = HashMap::from_iter([(source_id, shards)]); let delete_task = DeleteTask { diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs index 8627be9c5a6..072fa53dd03 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs @@ -21,11 +21,12 @@ use std::collections::HashMap; use itertools::Itertools; use quickwit_doc_mapper::{BinaryFormat, FieldMappingType}; +use quickwit_proto::ingest::Shard; use quickwit_proto::metastore::SourceType; use quickwit_proto::types::SourceId; use serde::{Deserialize, Serialize}; -use super::shards::{SerdeShards, Shards}; +use super::shards::Shards; use crate::file_backed_metastore::file_backed_index::FileBackedIndex; use crate::metastore::DeleteTask; use crate::{IndexMetadata, Split}; @@ -62,7 +63,7 @@ pub(crate) struct FileBackedIndexV0_7 { splits: Vec, // TODO: Remove `skip_serializing_if` when we release ingest v2. #[serde(default, skip_serializing_if = "HashMap::is_empty")] - shards: HashMap, + shards: HashMap>, #[serde(default)] delete_tasks: Vec, } @@ -82,8 +83,8 @@ impl From for FileBackedIndexV0_7 { // Skip serializing empty shards since the feature is hidden and disabled by // default. This way, we can still modify the serialization format without worrying // about backward compatibility post `0.7`. - if shards.next_shard_id > 1 { - Some((source_id, SerdeShards::from(shards))) + if !shards.is_empty() { + Some((source_id, shards.into_shards_vec())) } else { None } @@ -133,27 +134,32 @@ impl From for FileBackedIndex { split.split_metadata.index_uid = index.metadata.index_uid.clone(); } } - let mut shards: HashMap = index + let mut per_source_shards: HashMap = index .shards .into_iter() - .map(|(source_id, serde_shards)| { + .map(|(source_id, shards_vec)| { let index_uid = index.metadata.index_uid.clone(); ( source_id.clone(), - Shards::from_serde_shards(index_uid, source_id, serde_shards), + Shards::from_shards_vec(index_uid, source_id, shards_vec), ) }) .collect(); // TODO: Remove this when we release ingest v2. for source in index.metadata.sources.values() { if source.source_type() == SourceType::IngestV2 - && !shards.contains_key(&source.source_id) + && !per_source_shards.contains_key(&source.source_id) { let index_uid = index.metadata.index_uid.clone(); let source_id = source.source_id.clone(); - shards.insert(source_id.clone(), Shards::empty(index_uid, source_id)); + per_source_shards.insert(source_id.clone(), Shards::empty(index_uid, source_id)); } } - Self::new(index.metadata, index.splits, shards, index.delete_tasks) + Self::new( + index.metadata, + index.splits, + per_source_shards, + index.delete_tasks, + ) } } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs index ff5c0a87bec..0160c750a18 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/shards.rs @@ -28,20 +28,19 @@ use quickwit_proto::metastore::{ OpenShardsSubrequest, OpenShardsSubresponse, }; use quickwit_proto::types::{queue_id, IndexUid, Position, ShardId, SourceId}; -use serde::{Deserialize, Serialize}; use tracing::{info, warn}; use crate::checkpoint::{PartitionId, SourceCheckpoint, SourceCheckpointDelta}; use crate::file_backed_metastore::MutationOccurred; +// TODO: Rename `SourceShards` /// Manages the shards of a source. #[derive(Clone, Eq, PartialEq)] pub(crate) struct Shards { index_uid: IndexUid, source_id: SourceId, checkpoint: SourceCheckpoint, - pub next_shard_id: ShardId, - pub shards: HashMap, + shards: HashMap, } impl fmt::Debug for Shards { @@ -50,6 +49,7 @@ impl fmt::Debug for Shards { .field("index_uid", &self.index_uid) .field("source_id", &self.source_id) .field("num_shards", &self.shards.len()) + .field("shards", &self.shards) .finish() } } @@ -60,45 +60,51 @@ impl Shards { index_uid, source_id, checkpoint: SourceCheckpoint::default(), - next_shard_id: 1, // `1` matches the PostgreSQL sequence min value. shards: HashMap::new(), } } - pub(super) fn from_serde_shards( + pub(super) fn from_shards_vec( index_uid: IndexUid, source_id: SourceId, - serde_shards: SerdeShards, + shards_vec: Vec, ) -> Self { + let mut shards: HashMap = HashMap::with_capacity(shards_vec.len()); let mut checkpoint = SourceCheckpoint::default(); - let mut shards = HashMap::with_capacity(serde_shards.shards.len()); - - for shard in serde_shards.shards { - checkpoint.add_partition( - PartitionId::from(shard.shard_id), - shard.publish_position_inclusive(), - ); - shards.insert(shard.shard_id, shard); + + for shard in shards_vec { + let shard_id = shard.shard_id().clone(); + let partition_id = PartitionId::from(shard_id.as_str()); + let position = shard.publish_position_inclusive().clone(); + checkpoint.add_partition(partition_id, position); + shards.insert(shard_id, shard); } Self { index_uid, source_id, checkpoint, - next_shard_id: serde_shards.next_shard_id, shards, } } - fn get_shard(&self, shard_id: ShardId) -> MetastoreResult<&Shard> { - self.shards.get(&shard_id).ok_or_else(|| { + pub fn into_shards_vec(self) -> Vec { + self.shards.into_values().collect() + } + + pub fn is_empty(&self) -> bool { + self.shards.is_empty() + } + + fn get_shard(&self, shard_id: &ShardId) -> MetastoreResult<&Shard> { + self.shards.get(shard_id).ok_or_else(|| { let queue_id = queue_id(self.index_uid.as_str(), &self.source_id, shard_id); MetastoreError::NotFound(EntityKind::Shard { queue_id }) }) } - fn get_shard_mut(&mut self, shard_id: ShardId) -> MetastoreResult<&mut Shard> { - self.shards.get_mut(&shard_id).ok_or_else(|| { + fn get_shard_mut(&mut self, shard_id: &ShardId) -> MetastoreResult<&mut Shard> { + self.shards.get_mut(shard_id).ok_or_else(|| { let queue_id = queue_id(self.index_uid.as_str(), &self.source_id, shard_id); MetastoreError::NotFound(EntityKind::Shard { queue_id }) }) @@ -110,29 +116,15 @@ impl Shards { ) -> MetastoreResult> { let mut mutation_occurred = false; - // When a response is lost, the control plane can "lag by one shard ID". - // The inconsistency should be resolved on the next attempt. - // - // `self.next_shard_id - 1` does not underflow because `self.next_shard_id` is always > 0. - let ok_next_shard_ids = self.next_shard_id - 1..=self.next_shard_id; - - if !ok_next_shard_ids.contains(&subrequest.next_shard_id) { - warn!( - "control plane state is inconsistent with that of the metastore, expected next \ - shard ID `{}`, got `{}`", - self.next_shard_id, subrequest.next_shard_id - ); - return Err(MetastoreError::InconsistentControlPlaneState); - } - - let entry = self.shards.entry(subrequest.next_shard_id); + let shard_id = subrequest.shard_id(); + let entry = self.shards.entry(shard_id.clone()); let shard = match entry { Entry::Occupied(entry) => entry.get().clone(), Entry::Vacant(entry) => { let shard = Shard { index_uid: self.index_uid.clone().into(), source_id: self.source_id.clone(), - shard_id: self.next_shard_id, + shard_id: Some(shard_id.clone()), shard_state: ShardState::Open as i32, leader_id: subrequest.leader_id.clone(), follower_id: subrequest.follower_id.clone(), @@ -141,12 +133,11 @@ impl Shards { }; mutation_occurred = true; entry.insert(shard.clone()); - self.next_shard_id = subrequest.next_shard_id + 1; info!( index_id=%self.index_uid.index_id(), source_id=%self.source_id, - shard_id=%shard.shard_id, + shard_id=%shard_id, leader_id=%shard.leader_id, follower_id=?shard.follower_id, "opened shard" @@ -155,14 +146,11 @@ impl Shards { } }; let opened_shards = vec![shard]; - let next_shard_id = self.next_shard_id; - let response = OpenShardsSubresponse { subrequest_id: subrequest.subrequest_id, index_uid: subrequest.index_uid, source_id: subrequest.source_id, opened_shards, - next_shard_id, }; if mutation_occurred { Ok(MutationOccurred::Yes(response)) @@ -213,7 +201,7 @@ impl Shards { ) -> MetastoreResult> { let mut mutation_occurred = false; for shard_id in subrequest.shard_ids { - if let Entry::Occupied(entry) = self.shards.entry(shard_id) { + if let Entry::Occupied(entry) = self.shards.entry(shard_id.clone()) { let shard = entry.get(); if !force && !shard.publish_position_inclusive().is_eof() { let message = format!("shard `{shard_id}` is not deletable"); @@ -222,7 +210,7 @@ impl Shards { info!( index_id=%self.index_uid.index_id(), source_id=%self.source_id, - shard_id=%shard.shard_id, + shard_id=%shard_id, "deleted shard", ); entry.remove(); @@ -241,7 +229,6 @@ impl Shards { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shards, - next_shard_id: self.next_shard_id, }; Ok(response) } @@ -263,11 +250,8 @@ impl Shards { let mut shard_ids = Vec::with_capacity(checkpoint_delta.num_partitions()); for (partition_id, partition_delta) in checkpoint_delta.iter() { - let shard_id = partition_id.as_u64().ok_or_else(|| { - let message = format!("invalid partition ID: expected a u64, got `{partition_id}`"); - MetastoreError::InvalidArgument { message } - })?; - let shard = self.get_shard(shard_id)?; + let shard_id = ShardId::from(partition_id.as_str()); + let shard = self.get_shard(&shard_id)?; if shard.publish_token() != publish_token { let message = "failed to apply checkpoint delta: invalid publish token".to_string(); @@ -281,7 +265,7 @@ impl Shards { .expect("delta compatibility should have been checked"); for (shard_id, publish_position_inclusive) in shard_ids { - let shard = self.get_shard_mut(shard_id).expect("shard should exist"); + let shard = self.get_shard_mut(&shard_id).expect("shard should exist"); if publish_position_inclusive.is_eof() { shard.shard_state = ShardState::Closed as i32; @@ -304,22 +288,6 @@ impl Shards { } } -/// The serialized representation of [`SourceShards`]. -#[derive(Clone, Debug, Serialize, Deserialize)] -pub(super) struct SerdeShards { - pub next_shard_id: ShardId, - pub shards: Vec, -} - -impl From for SerdeShards { - fn from(shards: Shards) -> Self { - Self { - next_shard_id: shards.next_shard_id, - shards: shards.shards.into_values().collect(), - } - } -} - #[cfg(test)] mod tests { use quickwit_proto::ingest::ShardState; @@ -336,9 +304,9 @@ mod tests { subrequest_id: 0, index_uid: index_uid.clone().into(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(1)), leader_id: "leader_id".to_string(), follower_id: None, - next_shard_id: 1, }; let MutationOccurred::Yes(subresponse) = shards.open_shards(subrequest.clone()).unwrap() else { @@ -351,14 +319,13 @@ mod tests { let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, index_uid.as_str()); assert_eq!(shard.source_id, source_id); - assert_eq!(shard.shard_id, 1); + assert_eq!(shard.shard_id(), ShardId::from(1)); assert_eq!(shard.shard_state(), ShardState::Open); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id, None); assert_eq!(shard.publish_position_inclusive(), Position::Beginning); - assert_eq!(shards.shards.get(&1).unwrap(), shard); - assert_eq!(shards.next_shard_id, 2); + assert_eq!(shards.shards.get(&ShardId::from(1)).unwrap(), shard); let MutationOccurred::No(subresponse) = shards.open_shards(subrequest).unwrap() else { panic!("Expected `MutationOccured::No`"); @@ -366,16 +333,15 @@ mod tests { assert_eq!(subresponse.opened_shards.len(), 1); let shard = &subresponse.opened_shards[0]; - assert_eq!(shards.shards.get(&1).unwrap(), shard); - assert_eq!(shards.next_shard_id, 2); + assert_eq!(shards.shards.get(&ShardId::from(1)).unwrap(), shard); let subrequest = OpenShardsSubrequest { subrequest_id: 0, index_uid: index_uid.clone().into(), source_id: source_id.clone(), + shard_id: Some(ShardId::from(2)), leader_id: "leader_id".to_string(), follower_id: Some("follower_id".to_string()), - next_shard_id: 2, }; let MutationOccurred::Yes(subresponse) = shards.open_shards(subrequest).unwrap() else { panic!("Expected `MutationOccured::No`"); @@ -387,25 +353,13 @@ mod tests { let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, index_uid.as_str()); assert_eq!(shard.source_id, source_id); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id(), ShardId::from(2)); assert_eq!(shard.shard_state(), ShardState::Open); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id.as_ref().unwrap(), "follower_id"); assert_eq!(shard.publish_position_inclusive(), Position::Beginning); - assert_eq!(shards.shards.get(&2).unwrap(), shard); - assert_eq!(shards.next_shard_id, 3); - - let subrequest = OpenShardsSubrequest { - subrequest_id: 0, - index_uid: index_uid.clone().into(), - source_id: source_id.clone(), - leader_id: "leader_id".to_string(), - follower_id: Some("follower_id".to_string()), - next_shard_id: 1, - }; - let error = shards.open_shards(subrequest).unwrap_err(); - assert_eq!(error, MetastoreError::InconsistentControlPlaneState); + assert_eq!(shards.shards.get(&ShardId::from(2)).unwrap(), shard); } #[test] @@ -427,19 +381,19 @@ mod tests { let shard_0 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 0, + shard_id: Some(ShardId::from(0)), shard_state: ShardState::Open as i32, ..Default::default() }; let shard_1 = Shard { index_uid: index_uid.clone().into(), source_id: source_id.clone(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), shard_state: ShardState::Closed as i32, ..Default::default() }; - shards.shards.insert(0, shard_0); - shards.shards.insert(1, shard_1); + shards.shards.insert(ShardId::from(0), shard_0); + shards.shards.insert(ShardId::from(1), shard_1); let subrequest = ListShardsSubrequest { index_uid: index_uid.clone().into(), @@ -449,10 +403,10 @@ mod tests { let mut subresponse = shards.list_shards(subrequest).unwrap(); subresponse .shards - .sort_unstable_by_key(|shard| shard.shard_id); + .sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); assert_eq!(subresponse.shards.len(), 2); - assert_eq!(subresponse.shards[0].shard_id, 0); - assert_eq!(subresponse.shards[1].shard_id, 1); + assert_eq!(subresponse.shards[0].shard_id(), ShardId::from(0)); + assert_eq!(subresponse.shards[1].shard_id(), ShardId::from(1)); let subrequest = ListShardsSubrequest { index_uid: index_uid.into(), @@ -461,6 +415,6 @@ mod tests { }; let subresponse = shards.list_shards(subrequest).unwrap(); assert_eq!(subresponse.shards.len(), 1); - assert_eq!(subresponse.shards[0].shard_id, 1); + assert_eq!(subresponse.shards[0].shard_id(), ShardId::from(1)); } } diff --git a/quickwit/quickwit-metastore/src/tests/shard.rs b/quickwit/quickwit-metastore/src/tests/shard.rs index 54018d0e5a1..9cb0b6e2e54 100644 --- a/quickwit/quickwit-metastore/src/tests/shard.rs +++ b/quickwit/quickwit-metastore/src/tests/shard.rs @@ -23,7 +23,7 @@ use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, EntityKind, MetastoreError, MetastoreService, OpenShardsRequest, OpenShardsSubrequest, }; -use quickwit_proto::types::{IndexUid, SourceId}; +use quickwit_proto::types::{IndexUid, ShardId, SourceId}; use super::DefaultForTest; use crate::tests::cleanup_index; @@ -117,7 +117,6 @@ pub async fn test_metastore_open_shards< index_uid: "index-does-not-exist:0".to_string(), source_id: test_index.source_config.source_id.clone(), leader_id: "test-ingester-foo".to_string(), - next_shard_id: 1, ..Default::default() }], }; @@ -135,7 +134,6 @@ pub async fn test_metastore_open_shards< index_uid: test_index.index_uid.clone().into(), source_id: "source-does-not-exist".to_string(), leader_id: "test-ingester-foo".to_string(), - next_shard_id: 1, ..Default::default() }], }; @@ -152,8 +150,8 @@ pub async fn test_metastore_open_shards< subrequests: vec![OpenShardsSubrequest { index_uid: test_index.index_uid.clone().into(), source_id: test_index.source_config.source_id.clone(), + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester-foo".to_string(), - next_shard_id: 1, ..Default::default() }], }; @@ -163,13 +161,12 @@ pub async fn test_metastore_open_shards< let subresponse = &open_shards_response.subresponses[0]; assert_eq!(subresponse.index_uid, test_index.index_uid.as_str()); assert_eq!(subresponse.source_id, test_index.source_config.source_id); - assert_eq!(subresponse.next_shard_id, 2); assert_eq!(subresponse.opened_shards.len(), 1); let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, test_index.index_uid.as_str()); assert_eq!(shard.source_id, test_index.source_config.source_id); - assert_eq!(shard.shard_id, 1); + assert_eq!(shard.shard_id(), ShardId::from(1)); assert_eq!(shard.leader_id, "test-ingester-foo"); // Test open shard #1 is idempotent. @@ -177,8 +174,8 @@ pub async fn test_metastore_open_shards< subrequests: vec![OpenShardsSubrequest { index_uid: test_index.index_uid.clone().into(), source_id: test_index.source_config.source_id.clone(), + shard_id: Some(ShardId::from(1)), leader_id: "test-ingester-bar".to_string(), - next_shard_id: 1, ..Default::default() }], }; @@ -186,11 +183,10 @@ pub async fn test_metastore_open_shards< assert_eq!(open_shards_response.subresponses.len(), 1); let subresponse = &open_shards_response.subresponses[0]; - assert_eq!(subresponse.next_shard_id, 2); assert_eq!(subresponse.opened_shards.len(), 1); let shard = &subresponse.opened_shards[0]; - assert_eq!(shard.shard_id, 1); + assert_eq!(shard.shard_id(), ShardId::from(1)); assert_eq!(shard.leader_id, "test-ingester-foo"); // Test open shard #2. @@ -198,8 +194,8 @@ pub async fn test_metastore_open_shards< subrequests: vec![OpenShardsSubrequest { index_uid: test_index.index_uid.clone().into(), source_id: test_index.source_config.source_id.clone(), + shard_id: Some(ShardId::from(2)), leader_id: "test-ingester-qux".to_string(), - next_shard_id: 2, ..Default::default() }], }; @@ -209,34 +205,14 @@ pub async fn test_metastore_open_shards< let subresponse = &open_shards_response.subresponses[0]; assert_eq!(subresponse.index_uid, test_index.index_uid.as_str()); assert_eq!(subresponse.source_id, test_index.source_config.source_id); - assert_eq!(subresponse.next_shard_id, 3); assert_eq!(subresponse.opened_shards.len(), 1); let shard = &subresponse.opened_shards[0]; assert_eq!(shard.index_uid, test_index.index_uid.as_str()); assert_eq!(shard.source_id, test_index.source_config.source_id); - assert_eq!(shard.shard_id, 2); + assert_eq!(shard.shard_id(), ShardId::from(2)); assert_eq!(shard.leader_id, "test-ingester-qux"); - // Test open shard should not be called with a lagging `next_shard_id`. - let open_shards_request = OpenShardsRequest { - subrequests: vec![OpenShardsSubrequest { - index_uid: test_index.index_uid.clone().into(), - source_id: test_index.source_config.source_id.clone(), - leader_id: "test-ingester-foo".to_string(), - next_shard_id: 1, - ..Default::default() - }], - }; - let error = metastore - .open_shards(open_shards_request) - .await - .unwrap_err(); - assert!(matches!( - error, - MetastoreError::InconsistentControlPlaneState - )); - cleanup_index(&mut metastore, test_index.index_uid).await; } diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.expected.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.expected.json index 60c52413e18..0585a3519d5 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.expected.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.expected.json @@ -142,21 +142,6 @@ ], "version": "0.7" }, - "shards": { - "_ingest-source": { - "next_shard_id": 2, - "shards": [ - { - "follower_id": "follower-ingester", - "index_uid": "my-index:00000000000000000000000000", - "leader_id": "leader-ingester", - "shard_id": 1, - "shard_state": 0, - "source_id": "_ingest-source" - } - ] - } - }, "splits": [ { "create_timestamp": 3, diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.json index c8a478a3c3e..51cb6ecd3e8 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.6.json @@ -142,21 +142,6 @@ ], "version": "0.6" }, - "shards": { - "_ingest-source": { - "next_shard_id": 2, - "shards": [ - { - "follower_id": "follower-ingester", - "index_uid": "my-index:00000000000000000000000000", - "leader_id": "leader-ingester", - "shard_id": 1, - "shard_state": 0, - "source_id": "_ingest-source" - } - ] - } - }, "splits": [ { "create_timestamp": 3, diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.expected.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.expected.json index 60c52413e18..6ab7aefdff1 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.expected.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.expected.json @@ -143,19 +143,17 @@ "version": "0.7" }, "shards": { - "_ingest-source": { - "next_shard_id": 2, - "shards": [ - { - "follower_id": "follower-ingester", - "index_uid": "my-index:00000000000000000000000000", - "leader_id": "leader-ingester", - "shard_id": 1, - "shard_state": 0, - "source_id": "_ingest-source" - } - ] - } + "_ingest-source": [ + { + "index_uid": "my-index:00000000000000000000000000", + "source_id": "_ingest-source", + "shard_id": "00000000000000000001", + "shard_state": 1, + "leader_id": "leader-ingester", + "follower_id": "follower-ingester", + "publish_position_inclusive": "" + } + ] }, "splits": [ { diff --git a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.json b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.json index 60c52413e18..6ab7aefdff1 100644 --- a/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.json +++ b/quickwit/quickwit-metastore/test-data/file-backed-index/v0.7.json @@ -143,19 +143,17 @@ "version": "0.7" }, "shards": { - "_ingest-source": { - "next_shard_id": 2, - "shards": [ - { - "follower_id": "follower-ingester", - "index_uid": "my-index:00000000000000000000000000", - "leader_id": "leader-ingester", - "shard_id": 1, - "shard_state": 0, - "source_id": "_ingest-source" - } - ] - } + "_ingest-source": [ + { + "index_uid": "my-index:00000000000000000000000000", + "source_id": "_ingest-source", + "shard_id": "00000000000000000001", + "shard_state": 1, + "leader_id": "leader-ingester", + "follower_id": "follower-ingester", + "publish_position_inclusive": "" + } + ] }, "splits": [ { diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index 837dd53906f..7dca8722aa7 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -38,14 +38,17 @@ fn main() -> Result<(), Box> { // Indexing Service. let mut prost_config = prost_build::Config::default(); - prost_config.extern_path( - ".quickwit.indexing.PipelineUid", - "crate::types::PipelineUid", - ); + prost_config + .extern_path( + ".quickwit.indexing.PipelineUid", + "crate::types::PipelineUid", + ) + .extern_path(".quickwit.ingest.ShardId", "crate::types::ShardId"); Codegen::builder() .with_prost_config(prost_config) .with_protos(&["protos/quickwit/indexing.proto"]) + .with_includes(&["protos"]) .with_output_dir("src/codegen/quickwit") .with_result_type_path("crate::indexing::IndexingResult") .with_error_type_path("crate::indexing::IndexingError") @@ -55,6 +58,7 @@ fn main() -> Result<(), Box> { // Metastore service. let mut prost_config = prost_build::Config::default(); prost_config + .extern_path(".quickwit.ingest.ShardId", "crate::types::ShardId") .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") .field_attribute("DeleteQuery.query_ast", "#[serde(alias = \"query\")]") .field_attribute( @@ -87,6 +91,7 @@ fn main() -> Result<(), Box> { "Position.position", ]) .extern_path(".quickwit.ingest.Position", "crate::types::Position") + .extern_path(".quickwit.ingest.ShardId", "crate::types::ShardId") .type_attribute("Shard", "#[derive(Eq)]") .field_attribute( "Shard.follower_id", diff --git a/quickwit/quickwit-proto/protos/quickwit/indexing.proto b/quickwit/quickwit-proto/protos/quickwit/indexing.proto index f5bfcfae700..7f49bf7711c 100644 --- a/quickwit/quickwit-proto/protos/quickwit/indexing.proto +++ b/quickwit/quickwit-proto/protos/quickwit/indexing.proto @@ -21,6 +21,8 @@ syntax = "proto3"; package quickwit.indexing; +import "quickwit/ingest.proto"; + service IndexingService { // Apply an indexing plan on the node. rpc ApplyIndexingPlan(ApplyIndexingPlanRequest) returns (ApplyIndexingPlanResponse); @@ -42,7 +44,7 @@ message IndexingTask { // pipeline id PipelineUid pipeline_uid = 4; // The shards assigned to the indexer. - repeated uint64 shard_ids = 3; + repeated quickwit.ingest.ShardId shard_ids = 3; } message ApplyIndexingPlanResponse {} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto index 629dd28a981..a95446f5475 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingest.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -29,6 +29,14 @@ message Position { bytes position = 1; } +// The corresponding Rust struct [`crate::types::ShardId`] is defined manually and +// externally provided during code generation (see `build.rs`). +// +// Modify at your own risk. +message ShardId { + bytes shard_id = 1; +} + enum CommitTypeV2 { COMMIT_TYPE_V2_UNSPECIFIED = 0; COMMIT_TYPE_V2_AUTO = 1; @@ -63,7 +71,7 @@ message Shard { // Immutable fields string index_uid = 1; string source_id = 2; - uint64 shard_id = 3; + ShardId shard_id = 3; // The node ID of the ingester to which all the write requests for this shard should be sent to. string leader_id = 4; // The node ID of the ingester holding a copy of the data. @@ -83,5 +91,5 @@ message Shard { message ShardIds { string index_uid = 1; string source_id = 2; - repeated uint64 shard_ids = 3; + repeated ShardId shard_ids = 3; } diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 83fa424c016..694a32be1b6 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -61,7 +61,7 @@ service IngesterService { message RetainShardsForSource { string index_uid = 1; string source_id = 2; - repeated uint64 shard_ids = 3; + repeated quickwit.ingest.ShardId shard_ids = 3; } message RetainShardsRequest { @@ -81,7 +81,7 @@ message PersistSubrequest { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.DocBatchV2 doc_batch = 5; } @@ -95,7 +95,7 @@ message PersistSuccess { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.Position replication_position_inclusive = 5; } @@ -111,7 +111,7 @@ message PersistFailure { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; PersistFailureReason reason = 5; } @@ -165,7 +165,7 @@ message ReplicateSubrequest { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.Position from_position_exclusive = 5; quickwit.ingest.Position to_position_inclusive = 6; ingest.DocBatchV2 doc_batch = 7; @@ -183,7 +183,7 @@ message ReplicateSuccess { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.Position replication_position_inclusive = 5; } @@ -199,7 +199,7 @@ message ReplicateFailure { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; ReplicateFailureReason reason = 5; } @@ -211,7 +211,7 @@ message TruncateShardsRequest { message TruncateShardsSubrequest { string index_uid = 1; string source_id = 2; - uint64 shard_id = 3; + quickwit.ingest.ShardId shard_id = 3; // The position up to which the shard should be truncated (inclusive). quickwit.ingest.Position truncate_up_to_position_inclusive = 4; } @@ -224,7 +224,7 @@ message OpenFetchStreamRequest { string client_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; quickwit.ingest.Position from_position_exclusive = 5; } @@ -238,7 +238,7 @@ message FetchMessage { message FetchPayload { string index_uid = 1; string source_id = 2; - uint64 shard_id = 3; + quickwit.ingest.ShardId shard_id = 3; quickwit.ingest.MRecordBatch mrecord_batch = 4; quickwit.ingest.Position from_position_exclusive = 5; quickwit.ingest.Position to_position_inclusive = 6; @@ -247,7 +247,7 @@ message FetchPayload { message FetchEof { string index_uid = 1; string source_id = 2; - uint64 shard_id = 3; + quickwit.ingest.ShardId shard_id = 3; quickwit.ingest.Position eof_position = 4; } diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index c950dcaeeeb..f40181b8fa3 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -323,9 +323,9 @@ message OpenShardsSubrequest { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - string leader_id = 4; - optional string follower_id = 5; - uint64 next_shard_id = 6; + quickwit.ingest.ShardId shard_id = 4; + string leader_id = 5; + optional string follower_id = 6; } message OpenShardsResponse { @@ -337,7 +337,6 @@ message OpenShardsSubresponse { string index_uid = 2; string source_id = 3; repeated quickwit.ingest.Shard opened_shards = 4; - uint64 next_shard_id = 5; } message AcquireShardsRequest { @@ -347,7 +346,7 @@ message AcquireShardsRequest { message AcquireShardsSubrequest { string index_uid = 1; string source_id = 2; - repeated uint64 shard_ids = 3; + repeated quickwit.ingest.ShardId shard_ids = 3; string publish_token = 4; } @@ -370,7 +369,7 @@ message DeleteShardsRequest { message DeleteShardsSubrequest { string index_uid = 1; string source_id = 2; - repeated uint64 shard_ids = 3; + repeated quickwit.ingest.ShardId shard_ids = 3; } message DeleteShardsResponse { @@ -394,5 +393,4 @@ message ListShardsSubresponse { string index_uid = 1; string source_id = 2; repeated quickwit.ingest.Shard shards = 3; - uint64 next_shard_id = 4; } diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto index 54969f1e386..be34dc5df3a 100644 --- a/quickwit/quickwit-proto/protos/quickwit/router.proto +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -53,7 +53,7 @@ message IngestSuccess { uint32 subrequest_id = 1; string index_uid = 2; string source_id = 3; - uint64 shard_id = 4; + quickwit.ingest.ShardId shard_id = 4; // Replication position inclusive. quickwit.ingest.Position replication_position_inclusive = 5; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index d4c7b920a40..bcb1b548d51 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -19,8 +19,8 @@ pub struct IndexingTask { #[prost(message, optional, tag = "4")] pub pipeline_uid: ::core::option::Option, /// The shards assigned to the indexer. - #[prost(uint64, repeated, tag = "3")] - pub shard_ids: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub shard_ids: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] 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 3ee80664d78..43bd2fe7fb8 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -6,8 +6,8 @@ pub struct RetainShardsForSource { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, repeated, tag = "3")] - pub shard_ids: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub shard_ids: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -41,8 +41,8 @@ pub struct PersistSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub doc_batch: ::core::option::Option, } @@ -67,8 +67,8 @@ pub struct PersistSuccess { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } @@ -82,8 +82,8 @@ pub struct PersistFailure { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(enumeration = "PersistFailureReason", tag = "5")] pub reason: i32, } @@ -193,8 +193,8 @@ pub struct ReplicateSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub from_position_exclusive: ::core::option::Option, #[prost(message, optional, tag = "6")] @@ -226,8 +226,8 @@ pub struct ReplicateSuccess { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, } @@ -241,8 +241,8 @@ pub struct ReplicateFailure { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(enumeration = "ReplicateFailureReason", tag = "5")] pub reason: i32, } @@ -263,8 +263,8 @@ pub struct TruncateShardsSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, + #[prost(message, optional, tag = "3")] + pub shard_id: ::core::option::Option, /// The position up to which the shard should be truncated (inclusive). #[prost(message, optional, tag = "4")] pub truncate_up_to_position_inclusive: ::core::option::Option< @@ -286,8 +286,8 @@ pub struct OpenFetchStreamRequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "5")] pub from_position_exclusive: ::core::option::Option, } @@ -319,8 +319,8 @@ pub struct FetchPayload { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, + #[prost(message, optional, tag = "3")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "4")] pub mrecord_batch: ::core::option::Option, #[prost(message, optional, tag = "5")] @@ -336,8 +336,8 @@ pub struct FetchEof { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, + #[prost(message, optional, tag = "3")] + pub shard_id: ::core::option::Option, #[prost(message, optional, tag = "4")] pub eof_position: ::core::option::Option, } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index b05ce2f8d06..8362d858d9e 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -42,8 +42,8 @@ pub struct IngestSuccess { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "4")] - pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, /// Replication position inclusive. #[prost(message, optional, tag = "5")] pub replication_position_inclusive: ::core::option::Option, diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index 744131883ab..cf39091d9ae 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -28,8 +28,8 @@ pub struct Shard { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, + #[prost(message, optional, tag = "3")] + pub shard_id: ::core::option::Option, /// The node ID of the ingester to which all the write requests for this shard should be sent to. #[prost(string, tag = "4")] pub leader_id: ::prost::alloc::string::String, @@ -60,8 +60,8 @@ pub struct ShardIds { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, repeated, tag = "3")] - pub shard_ids: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub shard_ids: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 039a3ca9180..9bf5c0e580a 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -263,12 +263,12 @@ pub struct OpenShardsSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "3")] pub source_id: ::prost::alloc::string::String, - #[prost(string, tag = "4")] + #[prost(message, optional, tag = "4")] + pub shard_id: ::core::option::Option, + #[prost(string, tag = "5")] pub leader_id: ::prost::alloc::string::String, - #[prost(string, optional, tag = "5")] + #[prost(string, optional, tag = "6")] pub follower_id: ::core::option::Option<::prost::alloc::string::String>, - #[prost(uint64, tag = "6")] - pub next_shard_id: u64, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -289,8 +289,6 @@ pub struct OpenShardsSubresponse { pub source_id: ::prost::alloc::string::String, #[prost(message, repeated, tag = "4")] pub opened_shards: ::prost::alloc::vec::Vec, - #[prost(uint64, tag = "5")] - pub next_shard_id: u64, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -307,8 +305,8 @@ pub struct AcquireShardsSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, repeated, tag = "3")] - pub shard_ids: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub shard_ids: ::prost::alloc::vec::Vec, #[prost(string, tag = "4")] pub publish_token: ::prost::alloc::string::String, } @@ -348,8 +346,8 @@ pub struct DeleteShardsSubrequest { pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] pub source_id: ::prost::alloc::string::String, - #[prost(uint64, repeated, tag = "3")] - pub shard_ids: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + pub shard_ids: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -390,8 +388,6 @@ pub struct ListShardsSubresponse { pub source_id: ::prost::alloc::string::String, #[prost(message, repeated, tag = "3")] pub shards: ::prost::alloc::vec::Vec, - #[prost(uint64, tag = "4")] - pub next_shard_id: u64, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index c843e0e3be6..8efd0b9376f 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -340,7 +340,7 @@ impl Event for ShardPositionsUpdate {} impl IndexingTask { pub fn pipeline_uid(&self) -> PipelineUid { self.pipeline_uid - .expect("Pipeline UID should always be present.") + .expect("`pipeline_uid` should be a required field") } } diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs index 5b1f7c40746..f63859620dd 100644 --- a/quickwit/quickwit-proto/src/ingest/ingester.rs +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -17,12 +17,20 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use crate::types::{queue_id, Position, QueueId}; +use crate::types::{queue_id, Position, QueueId, ShardId}; include!("../codegen/quickwit/quickwit.ingest.ingester.rs"); pub use ingester_service_grpc_server::IngesterServiceGrpcServer; +impl FetchEof { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } +} + impl FetchMessage { pub fn new_payload(payload: FetchPayload) -> Self { assert!( @@ -48,8 +56,14 @@ impl FetchMessage { } impl FetchPayload { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } pub fn num_mrecords(&self) -> usize { @@ -60,40 +74,74 @@ impl FetchPayload { } } - pub fn from_position_exclusive(&self) -> Position { - self.from_position_exclusive.clone().unwrap_or_default() + pub fn from_position_exclusive(&self) -> &Position { + self.from_position_exclusive + .as_ref() + .expect("`from_position_exclusive` should be a required field") } - pub fn to_position_inclusive(&self) -> Position { - self.to_position_inclusive.clone().unwrap_or_default() + pub fn to_position_inclusive(&self) -> &Position { + self.to_position_inclusive + .as_ref() + .expect("`to_position_inclusive` should be a required field") } } impl FetchEof { - pub fn eof_position(&self) -> Position { - self.eof_position.clone().unwrap_or_default() + pub fn eof_position(&self) -> &Position { + self.eof_position + .as_ref() + .expect("`eof_position` should be a required field") } } impl OpenFetchStreamRequest { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } - pub fn from_position_exclusive(&self) -> Position { - self.from_position_exclusive.clone().unwrap_or_default() + pub fn from_position_exclusive(&self) -> &Position { + self.from_position_exclusive + .as_ref() + .expect("`from_position_exclusive` should be a required field") } } impl PersistSubrequest { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } } impl PersistSuccess { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) + } +} + +impl PersistFailure { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") } } @@ -164,35 +212,65 @@ impl AckReplicationMessage { } impl ReplicateSubrequest { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } - pub fn from_position_exclusive(&self) -> Position { - self.from_position_exclusive.clone().unwrap_or_default() + pub fn from_position_exclusive(&self) -> &Position { + self.from_position_exclusive + .as_ref() + .expect("`from_position_exclusive` should be a required field") } - pub fn to_position_inclusive(&self) -> Position { - self.to_position_inclusive.clone().unwrap_or_default() + pub fn to_position_inclusive(&self) -> &Position { + self.to_position_inclusive + .as_ref() + .expect("`to_position_inclusive` should be a required field") } } impl ReplicateSuccess { - pub fn replication_position_inclusive(&self) -> Position { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + + pub fn replication_position_inclusive(&self) -> &Position { self.replication_position_inclusive - .clone() - .unwrap_or_default() + .as_ref() + .expect("`replication_position_inclusive` should be a required field") + } +} + +impl ReplicateFailure { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") } } impl TruncateShardsSubrequest { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } - pub fn truncate_up_to_position_inclusive(&self) -> Position { + pub fn truncate_up_to_position_inclusive(&self) -> &Position { self.truncate_up_to_position_inclusive - .clone() - .unwrap_or_default() + .as_ref() + .expect("`truncate_up_to_position_inclusive` should be a required field") } } diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index 54da072ad7d..f78e8e1c2ff 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -195,6 +195,12 @@ impl MRecordBatch { } impl Shard { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } + pub fn is_open(&self) -> bool { self.shard_state().is_open() } @@ -208,11 +214,13 @@ impl Shard { } pub fn queue_id(&self) -> super::types::QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) + queue_id(&self.index_uid, &self.source_id, self.shard_id()) } - pub fn publish_position_inclusive(&self) -> Position { - self.publish_position_inclusive.clone().unwrap_or_default() + pub fn publish_position_inclusive(&self) -> &Position { + self.publish_position_inclusive + .as_ref() + .expect("`publish_position_inclusive` should be a required field") } } @@ -253,7 +261,7 @@ impl ShardIds { pub fn queue_ids(&self) -> impl Iterator + '_ { self.shard_ids .iter() - .map(|shard_id| queue_id(&self.index_uid, &self.source_id, *shard_id)) + .map(|shard_id| queue_id(&self.index_uid, &self.source_id, shard_id)) } } diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index bcddfa77876..b90c7dc85a6 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -22,7 +22,7 @@ use std::fmt; use quickwit_common::retry::Retryable; use serde::{Deserialize, Serialize}; -use crate::types::{IndexId, IndexUid, QueueId, SourceId, SplitId}; +use crate::types::{IndexId, IndexUid, QueueId, ShardId, SourceId, SplitId}; use crate::{ServiceError, ServiceErrorCode}; pub mod events; @@ -112,9 +112,6 @@ pub enum MetastoreError { #[error("access forbidden: {message}")] Forbidden { message: String }, - #[error("control plane state is inconsistent with that of the metastore")] - InconsistentControlPlaneState, - #[error("internal error: {message}; cause: `{cause}`")] Internal { message: String, cause: String }, @@ -178,7 +175,6 @@ impl ServiceError for MetastoreError { Self::Db { .. } => ServiceErrorCode::Internal, Self::FailedPrecondition { .. } => ServiceErrorCode::BadRequest, Self::Forbidden { .. } => ServiceErrorCode::MethodNotAllowed, - Self::InconsistentControlPlaneState { .. } => ServiceErrorCode::BadRequest, Self::Internal { .. } => ServiceErrorCode::Internal, Self::InvalidArgument { .. } => ServiceErrorCode::BadRequest, Self::Io { .. } => ServiceErrorCode::Internal, @@ -306,3 +302,11 @@ impl ListIndexesMetadataRequest { } } } + +impl OpenShardsSubrequest { + pub fn shard_id(&self) -> &ShardId { + self.shard_id + .as_ref() + .expect("`shard_id` should be a required field") + } +} diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index 71059280e72..b5adc7aef7a 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -30,9 +30,11 @@ pub use ulid::Ulid; mod pipeline_uid; mod position; +mod shard_id; pub use pipeline_uid::PipelineUid; pub use position::Position; +pub use shard_id::ShardId; pub type IndexId = String; @@ -40,8 +42,6 @@ pub type SourceId = String; pub type SplitId = String; -pub type ShardId = u64; - pub type SubrequestId = u32; /// See the file `ingest.proto` for more details. @@ -50,16 +50,20 @@ pub type PublishToken = String; /// Uniquely identifies a shard and its underlying mrecordlog queue. pub type QueueId = String; // // -pub fn queue_id(index_uid: &str, source_id: &str, shard_id: u64) -> QueueId { - format!("{}/{}/{}", index_uid, source_id, shard_id) +pub fn queue_id(index_uid: &str, source_id: &str, shard_id: &ShardId) -> QueueId { + format!("{index_uid}/{source_id}/{shard_id}") } pub fn split_queue_id(queue_id: &str) -> Option<(IndexUid, SourceId, ShardId)> { let mut parts = queue_id.split('/'); let index_uid = parts.next()?; let source_id = parts.next()?; - let shard_id = parts.next()?.parse::().ok()?; - Some((index_uid.into(), source_id.to_string(), shard_id)) + let shard_id = parts.next()?; + Some(( + index_uid.into(), + source_id.to_string(), + ShardId::from(shard_id), + )) } /// Index identifiers that uniquely identify not only the index, but also @@ -397,8 +401,8 @@ mod tests { #[test] fn test_queue_id() { assert_eq!( - queue_id("test-index:0", "test-source", 1), - "test-index:0/test-source/1" + queue_id("test-index:0", "test-source", &ShardId::from(1u64)), + "test-index:0/test-source/00000000000000000001" ); } @@ -410,14 +414,11 @@ mod tests { let splits = split_queue_id("test-index:0/test-source"); assert!(splits.is_none()); - let splits = split_queue_id("test-index:0/test-source/a"); - assert!(splits.is_none()); - let (index_uid, source_id, shard_id) = - split_queue_id("test-index:0/test-source/1").unwrap(); + split_queue_id("test-index:0/test-source/00000000000000000001").unwrap(); assert_eq!(index_uid, "test-index:0"); assert_eq!(source_id, "test-source"); - assert_eq!(shard_id, 1); + assert_eq!(shard_id, ShardId::from(1u64)); } #[test] diff --git a/quickwit/quickwit-proto/src/types/position.rs b/quickwit/quickwit-proto/src/types/position.rs index 0ef090f7eaa..e92a72475d1 100644 --- a/quickwit/quickwit-proto/src/types/position.rs +++ b/quickwit/quickwit-proto/src/types/position.rs @@ -223,6 +223,13 @@ impl<'de> Deserialize<'de> for Position { } } +impl PartialEq for &Position { + #[inline] + fn eq(&self, other: &Position) -> bool { + *self == other + } +} + impl prost::Message for Position { fn encode_raw(&self, buf: &mut B) where B: prost::bytes::BufMut { diff --git a/quickwit/quickwit-proto/src/types/shard_id.rs b/quickwit/quickwit-proto/src/types/shard_id.rs new file mode 100644 index 00000000000..9d31ce5e77f --- /dev/null +++ b/quickwit/quickwit-proto/src/types/shard_id.rs @@ -0,0 +1,168 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::fmt; +use std::fmt::Debug; + +use bytestring::ByteString; +use prost::DecodeError; +use serde::{Deserialize, Serialize}; +use ulid::Ulid; + +/// Shard ID. +#[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Ord, PartialOrd)] +pub struct ShardId(ByteString); + +impl ShardId { + pub fn as_str(&self) -> &str { + &self.0 + } + + pub fn as_u64(&self) -> Option { + self.0.parse().ok() + } +} + +impl fmt::Display for ShardId { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", &self.0) + } +} + +impl From<&str> for ShardId { + fn from(shard_id: &str) -> Self { + Self(ByteString::from(shard_id)) + } +} + +impl From for ShardId { + fn from(shard_id: String) -> Self { + Self(ByteString::from(shard_id)) + } +} + +impl From for ShardId { + fn from(shard_id: u64) -> Self { + Self(ByteString::from(format!("{shard_id:0>20}"))) + } +} + +impl From for ShardId { + fn from(shard_id: Ulid) -> Self { + Self(ByteString::from(shard_id.to_string())) + } +} + +impl Serialize for ShardId { + fn serialize(&self, serializer: S) -> Result { + serializer.collect_str(self) + } +} + +impl<'de> Deserialize<'de> for ShardId { + fn deserialize>(deserializer: D) -> Result { + let shard_id = String::deserialize(deserializer)?; + Ok(Self::from(shard_id)) + } +} + +impl prost::Message for ShardId { + fn encode_raw(&self, buf: &mut B) + where B: prost::bytes::BufMut { + prost::encoding::bytes::encode(1u32, &self.0.as_bytes().clone(), buf); + } + + fn merge_field( + &mut self, + tag: u32, + wire_type: prost::encoding::WireType, + buf: &mut B, + ctx: prost::encoding::DecodeContext, + ) -> ::core::result::Result<(), prost::DecodeError> + where + B: prost::bytes::Buf, + { + const STRUCT_NAME: &str = "ShardId"; + + match tag { + 1u32 => { + let mut value = Vec::new(); + prost::encoding::bytes::merge(wire_type, &mut value, buf, ctx).map_err( + |mut error| { + error.push(STRUCT_NAME, "position"); + error + }, + )?; + let byte_string = ByteString::try_from(value) + .map_err(|_| DecodeError::new("shard_id is not valid UTF-8"))?; + *self = Self(byte_string); + Ok(()) + } + _ => prost::encoding::skip_field(wire_type, tag, buf, ctx), + } + } + + #[inline] + fn encoded_len(&self) -> usize { + prost::encoding::bytes::encoded_len(1u32, &self.0.as_bytes().clone()) + } + + fn clear(&mut self) { + *self = Self::default(); + } +} + +impl PartialEq for &ShardId { + #[inline] + fn eq(&self, other: &ShardId) -> bool { + *self == other + } +} + +#[cfg(test)] +mod tests { + + use bytes::Bytes; + use prost::Message; + + use super::*; + + #[test] + fn test_shard_id_json_serde_roundtrip() { + let serialized = serde_json::to_string(&ShardId::from(0)).unwrap(); + assert_eq!(serialized, r#""00000000000000000000""#); + let deserialized: ShardId = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, ShardId::from(0)); + } + + #[test] + fn test_shard_id_prost_serde_roundtrip() { + let ulid = Ulid::new(); + let encoded = ShardId::from(ulid).encode_to_vec(); + assert_eq!( + ShardId::decode(Bytes::from(encoded)).unwrap(), + ShardId::from(ulid) + ); + let encoded = ShardId::from(ulid).encode_length_delimited_to_vec(); + assert_eq!( + ShardId::decode_length_delimited(Bytes::from(encoded)).unwrap(), + ShardId::from(ulid) + ); + } +} diff --git a/quickwit/quickwit-serve/src/elastic_search_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elastic_search_api/bulk_v2.rs index 493d21ab230..6157b16aef8 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/bulk_v2.rs @@ -103,7 +103,7 @@ mod tests { use quickwit_proto::ingest::router::{ IngestFailure, IngestFailureReason, IngestResponseV2, IngestSuccess, }; - use quickwit_proto::types::Position; + use quickwit_proto::types::{Position, ShardId}; use warp::{Filter, Rejection, Reply}; use super::*; @@ -164,14 +164,14 @@ mod tests { subrequest_id: 0, index_uid: "my-index-1:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(1u64)), }, IngestSuccess { subrequest_id: 1, index_uid: "my-index-2:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), }, ], @@ -247,7 +247,7 @@ mod tests { subrequest_id: 0, index_uid: "my-index-1:0".to_string(), source_id: INGEST_V2_SOURCE_ID.to_string(), - shard_id: 1, + shard_id: Some(ShardId::from(1)), replication_position_inclusive: Some(Position::offset(0u64)), }], failures: Vec::new(),