diff --git a/docker-compose.yml b/docker-compose.yml index cfead637271..884cd1e83f1 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -208,7 +208,7 @@ services: # It is not an official docker image # if we prefer we can build a docker from the official docker image (gcloud cli) # and install the pubsub emulator https://cloud.google.com/pubsub/docs/emulator - image: thekevjames/gcloud-pubsub-emulator:${GCLOUD_EMULATOR:-7555256f2c} + image: thekevjames/gcloud-pubsub-emulator:${GCLOUD_EMULATOR:-455.0.0} container_name: gcp-pubsub-emulator ports: - "${MAP_HOST_GCLOUD_EMULATOR:-127.0.0.1}:8681:8681" diff --git a/quickwit/quickwit-cli/src/source.rs b/quickwit/quickwit-cli/src/source.rs index 2abfd95a1fa..cecb7586434 100644 --- a/quickwit/quickwit-cli/src/source.rs +++ b/quickwit/quickwit-cli/src/source.rs @@ -433,7 +433,7 @@ where .iter() .map(|(partition_id, position)| CheckpointRow { partition_id: partition_id.0.to_string(), - offset: position.as_str().to_string(), + offset: position.to_string(), }) .sorted_by(|left, right| left.partition_id.cmp(&right.partition_id)); let checkpoint_table = make_table("Checkpoint", checkpoint_rows, false); @@ -734,7 +734,9 @@ mod tests { let checkpoint: SourceCheckpoint = vec![("shard-000", ""), ("shard-001", "1234567890")] .into_iter() - .map(|(partition_id, offset)| (PartitionId::from(partition_id), Position::from(offset))) + .map(|(partition_id, offset)| { + (PartitionId::from(partition_id), Position::offset(offset)) + }) .collect(); let sources = vec![SourceConfig { source_id: "foo-source".to_string(), diff --git a/quickwit/quickwit-common/src/tower/rate_limit.rs b/quickwit/quickwit-common/src/tower/rate_limit.rs index 7a86f674404..b065cc9f6af 100644 --- a/quickwit/quickwit-common/src/tower/rate_limit.rs +++ b/quickwit/quickwit-common/src/tower/rate_limit.rs @@ -276,7 +276,9 @@ mod tests { .call(Request { cost: 1 }) .await .unwrap(); - assert!(now.elapsed() < Duration::from_millis(1)); + // The request should go through immediately but in some rare instance the test is slow to + // run and the call to `call` takes more than 1 ms. + assert!(now.elapsed() < Duration::from_millis(5)); let now = Instant::now(); // The first request goes through, but the second one is rate limited. diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index da7f9c6eda8..419df93f884 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -41,7 +41,7 @@ use quickwit_proto::metastore::{ DeleteSourceRequest, EmptyResponse, MetastoreError, MetastoreService, MetastoreServiceClient, ToggleSourceRequest, }; -use quickwit_proto::types::{IndexUid, NodeId, Position, ShardId, SourceUid}; +use quickwit_proto::types::{IndexUid, NodeId, ShardId, SourceUid}; use serde::Serialize; use tracing::error; @@ -195,9 +195,7 @@ impl Handler for ControlPlane { let shard_ids_to_close: Vec = shard_positions_update .shard_positions .into_iter() - .filter(|(shard_id, position)| { - (position == &Position::Eof) && known_shard_ids.contains(shard_id) - }) + .filter(|(shard_id, position)| position.is_eof() && known_shard_ids.contains(shard_id)) .map(|(shard_id, _position)| shard_id) .collect(); if shard_ids_to_close.is_empty() { @@ -538,6 +536,7 @@ mod tests { DeleteShardsResponse, EntityKind, ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, ListShardsSubresponse, MetastoreError, SourceType, }; + use quickwit_proto::types::Position; use super::*; use crate::IndexerNodeInfo; @@ -1114,7 +1113,7 @@ mod tests { control_plane_mailbox .ask(ShardPositionsUpdate { source_uid: source_uid.clone(), - shard_positions: vec![(17, 1000u64.into())], + shard_positions: vec![(17, Position::offset(1_000u64))], }) .await .unwrap(); @@ -1137,7 +1136,7 @@ mod tests { control_plane_mailbox .ask(ShardPositionsUpdate { source_uid, - shard_positions: vec![(17, Position::Eof)], + shard_positions: vec![(17, Position::eof(1_000u64))], }) .await .unwrap(); @@ -1240,7 +1239,7 @@ mod tests { control_plane_mailbox .ask(ShardPositionsUpdate { source_uid: source_uid.clone(), - shard_positions: vec![(17, Position::Eof)], + shard_positions: vec![(17, Position::eof(1_000u64))], }) .await .unwrap(); diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index 35a2184d005..2c9e4c3436b 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -271,7 +271,7 @@ mod tests { suggest_truncate_checkpoints[0] .position_for_partition(&PartitionId::default()) .unwrap(), - &Position::from(2u64) + &Position::offset(2u64) ); let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); @@ -344,7 +344,7 @@ mod tests { suggest_truncate_checkpoints[0] .position_for_partition(&PartitionId::default()) .unwrap(), - &Position::from(2u64) + &Position::offset(2u64) ); let merger_msgs: Vec = merge_planner_inbox.drain_for_test_typed::(); diff --git a/quickwit/quickwit-indexing/src/models/shard_positions.rs b/quickwit/quickwit-indexing/src/models/shard_positions.rs index 2d18276eb92..2148c47c1d6 100644 --- a/quickwit/quickwit-indexing/src/models/shard_positions.rs +++ b/quickwit/quickwit-indexing/src/models/shard_positions.rs @@ -356,23 +356,23 @@ mod tests { )); event_broker1.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, 10u64.into())], + vec![(2, Position::offset(10u64))], )); event_broker1.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, 10u64.into())], + vec![(1, Position::offset(10u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, 10u64.into())], + vec![(2, Position::offset(10u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(2, 12u64.into())], + vec![(2, Position::offset(12u64))], )); event_broker2.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, Position::Beginning), (2, 12u64.into())], + vec![(1, Position::Beginning), (2, Position::offset(12u64))], )); let mut updates1: Vec> = Vec::new(); @@ -387,9 +387,9 @@ mod tests { updates1, vec![ vec![(1, Position::Beginning)], - vec![(1, Position::Beginning), (2, 10u64.into())], - vec![(1, 10u64.into()), (2, 10u64.into()),], - vec![(1, 10u64.into()), (2, 12u64.into()),], + 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)),], ] ); @@ -403,10 +403,10 @@ mod tests { assert_eq!( updates2, vec![ - vec![(2, 10u64.into())], - vec![(2, 12u64.into())], - vec![(1, Position::Beginning), (2, 12u64.into())], - vec![(1, 10u64.into()), (2, 12u64.into())], + 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))], ] ); @@ -446,16 +446,19 @@ mod tests { source_uid.clone(), vec![(1, Position::Beginning)], )); - tokio::time::sleep(Duration::from_millis(1000)).await; + tokio::time::sleep(Duration::from_secs(1)).await; let value = cluster.get_self_key_value(&key).await.unwrap(); assert_eq!(&value, r#"{"1":""}"#); } { event_broker.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, 1_000u64.into()), (2, 2000u64.into())], + vec![ + (1, Position::offset(1_000u64)), + (2, Position::offset(2_000u64)), + ], )); - tokio::time::sleep(Duration::from_millis(1000)).await; + tokio::time::sleep(Duration::from_secs(1)).await; let value = cluster.get_self_key_value(&key).await.unwrap(); assert_eq!( &value, @@ -465,9 +468,12 @@ mod tests { { event_broker.publish(LocalShardPositionsUpdate::new( source_uid.clone(), - vec![(1, 999u64.into()), (3, 3000u64.into())], + vec![ + (1, Position::offset(999u64)), + (3, Position::offset(3_000u64)), + ], )); - tokio::time::sleep(Duration::from_millis(1000)).await; + tokio::time::sleep(Duration::from_secs(1)).await; let value = cluster.get_self_key_value(&key).await.unwrap(); // We do not update the position that got lower, nor the position that disappeared assert_eq!( diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index 40e57f163c3..fc614ca97a7 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -100,8 +100,8 @@ impl Source for FileSource { .checkpoint_delta .record_partition_delta( partition_id, - Position::from(self.counters.previous_offset), - Position::from(self.counters.current_offset), + Position::offset(self.counters.previous_offset), + Position::offset(self.counters.current_offset), ) .unwrap(); } @@ -364,8 +364,8 @@ mod tests { let partition_id = PartitionId::from(temp_file_path.to_string_lossy().to_string()); let checkpoint_delta = SourceCheckpointDelta::from_partition_delta( partition_id, - Position::from(0u64), - Position::from(4u64), + Position::offset(0u64), + Position::offset(4u64), ) .unwrap(); checkpoint.try_apply_delta(checkpoint_delta).unwrap(); diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 7c70931fa56..4449841bebb 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -22,9 +22,10 @@ use std::fmt; use std::sync::Arc; use std::time::Duration; -use anyhow::{bail, Context}; +use anyhow::Context; use async_trait::async_trait; use fnv::FnvHashMap; +use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_common::pubsub::EventBroker; use quickwit_common::retry::RetryParams; @@ -33,7 +34,8 @@ use quickwit_ingest::{ }; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::ingest::ingester::{ - FetchResponseV2, IngesterService, TruncateShardsRequest, TruncateShardsSubrequest, + fetch_message, FetchEof, FetchPayload, IngesterService, TruncateShardsRequest, + TruncateShardsSubrequest, }; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsSubrequest, AcquireShardsSubresponse, MetastoreService, @@ -42,6 +44,7 @@ use quickwit_proto::metastore::{ use quickwit_proto::types::{ IndexUid, NodeId, Position, PublishToken, ShardId, SourceId, SourceUid, }; +use serde::Serialize; use serde_json::json; use tokio::time; use tracing::{debug, error, info, warn}; @@ -112,13 +115,14 @@ impl ClientId { } } -#[derive(Debug, Clone, Copy, Default, Eq, PartialEq)] +#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Serialize)] +#[serde(rename_all = "snake_case")] enum IndexingStatus { #[default] Active, - // We have received all documents from the stream. Note they + // We have received all documents from the stream. Note that they // are not necessarily published yet. - EofReached, + ReachedEof, // All documents have been indexed AND published. Complete, Error, @@ -192,24 +196,28 @@ impl IngestSource { }) } - fn process_fetch_response( + fn process_fetch_payload( &mut self, batch_builder: &mut BatchBuilder, - fetch_response: FetchResponseV2, + fetch_payload: FetchPayload, ) -> anyhow::Result<()> { - let Some(mrecord_batch) = &fetch_response.mrecord_batch else { - return Ok(()); + let mrecord_batch = match &fetch_payload.mrecord_batch { + Some(mrecord_batch) if !mrecord_batch.is_empty() => mrecord_batch, + _ => { + warn!("received empty mrecord batch"); + return Ok(()); + } }; let assigned_shard = self .assigned_shards - .get_mut(&fetch_response.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_response.from_position_exclusive(); - let to_position_inclusive = fetch_response.to_position_inclusive(); + let from_position_exclusive = fetch_payload.from_position_exclusive(); + let to_position_inclusive = fetch_payload.to_position_inclusive(); for mrecord in decoded_mrecords(mrecord_batch) { match mrecord { @@ -219,13 +227,6 @@ impl IngestSource { MRecord::Commit => { batch_builder.force_commit(); } - MRecord::Eof => { - assigned_shard.status = IndexingStatus::EofReached; - break; - } - MRecord::Unknown => { - bail!("source cannot decode mrecord"); - } } } batch_builder @@ -240,12 +241,41 @@ impl IngestSource { Ok(()) } + fn process_fetch_eof( + &mut self, + batch_builder: &mut BatchBuilder, + fetch_eof: FetchEof, + ) -> anyhow::Result<()> { + let assigned_shard = self + .assigned_shards + .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(); + + batch_builder + .checkpoint_delta + .record_partition_delta( + partition_id, + from_position_exclusive, + to_position_inclusive.clone(), + ) + .context("failed to record partition delta")?; + assigned_shard.current_position_inclusive = to_position_inclusive; + Ok(()) + } + fn process_fetch_stream_error(&mut self, fetch_stream_error: FetchStreamError) { - if let Some(shard) = self.assigned_shards.get_mut(&fetch_stream_error.shard_id) { - if shard.status != IndexingStatus::Complete - || shard.status != IndexingStatus::EofReached - { - shard.status = IndexingStatus::Error; + if let Some(assigned_shard) = self.assigned_shards.get_mut(&fetch_stream_error.shard_id) { + if !matches!( + assigned_shard.status, + IndexingStatus::ReachedEof | IndexingStatus::Complete + ) { + assigned_shard.status = IndexingStatus::Error; } } } @@ -258,7 +288,7 @@ impl IngestSource { // Let's record all shards that have reached Eof as complete. for (shard, truncate_position) in &truncate_positions { - if truncate_position == &Position::Eof { + if truncate_position.is_eof() { if let Some(assigned_shard) = self.assigned_shards.get_mut(shard) { assigned_shard.status = IndexingStatus::Complete; } @@ -338,15 +368,6 @@ impl IngestSource { } } - fn contains_publish_token(&self, subresponse: &AcquireShardsSubresponse) -> bool { - if let Some(acquired_shard) = subresponse.acquired_shards.get(0) { - if let Some(publish_token) = &acquired_shard.publish_token { - return *publish_token == self.publish_token; - } - } - false - } - /// If the new assignment removes a shard that we were in the middle of indexing (ie they have /// not reached `IndexingStatus::Complete` status yet), we need to reset the pipeline: /// @@ -416,6 +437,15 @@ impl IngestSource { .await?; Ok(()) } + + fn contains_publish_token(&self, subresponse: &AcquireShardsSubresponse) -> bool { + if let Some(acquired_shard) = subresponse.acquired_shards.get(0) { + if let Some(publish_token) = &acquired_shard.publish_token { + return *publish_token == self.publish_token; + } + } + false + } } #[async_trait] @@ -432,13 +462,22 @@ impl Source for IngestSource { loop { match time::timeout_at(deadline, self.fetch_stream.next()).await { - Ok(Ok(fetch_payload)) => { - self.process_fetch_response(&mut batch_builder, fetch_payload)?; + Ok(Ok(fetch_message)) => match fetch_message.message { + Some(fetch_message::Message::Payload(fetch_payload)) => { + self.process_fetch_payload(&mut batch_builder, fetch_payload)?; - if batch_builder.num_bytes >= BATCH_NUM_BYTES_LIMIT { - break; + if batch_builder.num_bytes >= BATCH_NUM_BYTES_LIMIT { + break; + } } - } + Some(fetch_message::Message::Eof(fetch_eof)) => { + self.process_fetch_eof(&mut batch_builder, fetch_eof)?; + } + None => { + warn!("received empty fetch message"); + continue; + } + }, Ok(Err(fetch_stream_error)) => { self.process_fetch_stream_error(fetch_stream_error); } @@ -524,7 +563,7 @@ impl Source for IngestSource { .publish_position_inclusive .unwrap_or_default(); let from_position_exclusive = current_position_inclusive.clone(); - let status = if from_position_exclusive == Position::Eof { + let status = if from_position_exclusive.is_eof() { IndexingStatus::Complete } else if let Err(error) = ctx .protect_future(self.fetch_stream.subscribe( @@ -547,7 +586,7 @@ impl Source for IngestSource { leader_id, follower_id_opt, partition_id, - current_position_inclusive: current_position_inclusive.clone(), + current_position_inclusive, status, }; self.assigned_shards.insert(shard_id, assigned_shard); @@ -577,9 +616,21 @@ impl Source for IngestSource { } fn observable_state(&self) -> serde_json::Value { + let assigned_shards: Vec = self + .assigned_shards + .iter() + .sorted_by(|(left_shard_id, _), (right_shard_id, _)| left_shard_id.cmp(right_shard_id)) + .map(|(shard_id, assigned_shard)| { + json!({ + "shard_id": *shard_id, + "current_position": assigned_shard.current_position_inclusive, + "status": assigned_shard.status, + }) + }) + .collect(); json!({ "client_id": self.client_id.to_string(), - "assigned_shards": self.assigned_shards.keys().copied().collect::>(), + "assigned_shards": assigned_shards, "publish_token": self.publish_token, }) } @@ -590,13 +641,14 @@ mod tests { use std::iter::once; use std::path::PathBuf; - use bytes::Bytes; use itertools::Itertools; use quickwit_actors::{ActorContext, Universe}; use quickwit_common::ServiceStream; use quickwit_config::{SourceConfig, SourceParams}; use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::ingest::ingester::{IngesterServiceClient, TruncateShardsResponse}; + use quickwit_proto::ingest::ingester::{ + FetchMessage, IngesterServiceClient, TruncateShardsResponse, + }; use quickwit_proto::ingest::{IngestV2Error, MRecordBatch, Shard, ShardState}; use quickwit_proto::metastore::{AcquireShardsResponse, AcquireShardsSubresponse}; use quickwit_proto::types::PipelineUid; @@ -648,7 +700,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 0, shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(10u64.into()), + publish_position_inclusive: Some(Position::offset(10u64)), publish_token: Some(publish_token.to_string()), }], }], @@ -680,7 +732,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(11u64.into()), + publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), }], }], @@ -713,7 +765,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(11u64.into()), + publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), }, Shard { @@ -723,7 +775,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 2, shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(12u64.into()), + publish_position_inclusive: Some(Position::offset(12u64)), publish_token: Some(publish_token.to_string()), }, ], @@ -741,7 +793,10 @@ mod tests { let sequence_tx_clone1 = sequence_tx.clone(); ingester_mock_0 .expect_open_fetch_stream() - .withf(|req| req.from_position_exclusive == Some(10u64.into()) && req.shard_id == 0) + .withf(|request| { + request.from_position_exclusive() == Position::offset(10u64) + && request.shard_id == 0 + }) .once() .returning(move |request| { sequence_tx_clone1.send(1).unwrap(); @@ -751,13 +806,17 @@ mod tests { ); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); + let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); Ok(service_stream) }); let sequence_tx_clone2 = sequence_tx.clone(); ingester_mock_0 .expect_open_fetch_stream() - .withf(|req| req.from_position_exclusive == Some(11u64.into()) && req.shard_id == 1) + .withf(|request| { + request.from_position_exclusive() == Position::offset(11u64) + && request.shard_id == 1 + }) .times(2) .returning(move |request| { sequence_tx_clone2.send(2).unwrap(); @@ -767,13 +826,17 @@ mod tests { ); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); + let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); Ok(service_stream) }); let sequence_tx_clone3 = sequence_tx.clone(); ingester_mock_0 .expect_open_fetch_stream() - .withf(|req| req.from_position_exclusive == Some(12u64.into()) && req.shard_id == 2) + .withf(|request| { + request.from_position_exclusive() == Position::offset(12u64) + && request.shard_id == 2 + }) .once() .returning(move |request| { sequence_tx_clone3.send(3).unwrap(); @@ -783,6 +846,7 @@ mod tests { ); assert_eq!(request.index_uid, "test-index:0"); assert_eq!(request.source_id, "test-source"); + let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); Ok(service_stream) }); @@ -797,7 +861,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.to_position_inclusive, Some(10u64.into())); + assert_eq!(subrequest.to_position_inclusive(), Position::offset(10u64)); let response = TruncateShardsResponse {}; Ok(response) @@ -810,10 +874,12 @@ mod tests { .returning(|request| { assert_eq!(request.ingester_id, "test-ingester-0"); assert_eq!(request.subrequests.len(), 1); + let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.to_position_inclusive, Some(11u64.into())); + assert_eq!(subrequest.to_position_inclusive(), Position::offset(11u64)); + Ok(TruncateShardsResponse {}) }); ingester_mock_0 @@ -830,12 +896,12 @@ 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.to_position_inclusive, Some(11u64.into())); + assert_eq!(subrequest.to_position_inclusive(), Position::offset(11u64)); let subrequest = &request.subrequests[1]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.to_position_inclusive, Some(12u64.into())); + assert_eq!(subrequest.to_position_inclusive(), Position::offset(12u64)); let response = TruncateShardsResponse {}; Ok(response) @@ -934,7 +1000,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: 11u64.into(), + current_position_inclusive: Position::offset(11u64), status: IndexingStatus::Active, }; assert_eq!(assigned_shard, &expected_assigned_shard); @@ -944,7 +1010,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 2u64.into(), - current_position_inclusive: 12u64.into(), + current_position_inclusive: Position::offset(12u64), status: IndexingStatus::Active, }; assert_eq!(assigned_shard, &expected_assigned_shard); @@ -979,22 +1045,34 @@ 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, vec![1, 2]); let response = AcquireShardsResponse { subresponses: vec![AcquireShardsSubresponse { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - acquired_shards: vec![Shard { - leader_id: "test-ingester-0".to_string(), - follower_id: None, - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(Position::Eof), - publish_token: Some(publish_token.to_string()), - }], + acquired_shards: vec![ + Shard { + leader_id: "test-ingester-0".to_string(), + follower_id: None, + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + shard_state: ShardState::Open as i32, + publish_position_inclusive: Some(Position::eof(11u64)), + publish_token: Some(publish_token.to_string()), + }, + Shard { + leader_id: "test-ingester-0".to_string(), + follower_id: None, + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 2, + shard_state: ShardState::Open as i32, + publish_position_inclusive: Some(Position::Beginning.as_eof()), + publish_token: Some(publish_token.to_string()), + }, + ], }], }; Ok(response) @@ -1007,13 +1085,22 @@ mod tests { .once() .returning(|request| { assert_eq!(request.ingester_id, "test-ingester-0"); - assert_eq!(request.subrequests.len(), 1); + assert_eq!(request.subrequests.len(), 2); - 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.to_position_inclusive, Some(Position::Eof)); + 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.to_position_inclusive(), Position::eof(11u64)); + + 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.to_position_inclusive(), + Position::Beginning.as_eof() + ); let response = TruncateShardsResponse {}; Ok(response) @@ -1053,8 +1140,8 @@ mod tests { let ctx: SourceContext = 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 = once(1).collect(); + // In this scenario, the indexer will be able to acquire shard 1 and 2. + let shard_ids: BTreeSet = BTreeSet::from_iter([1, 2]); source .assign_shards(shard_ids, &doc_processor_mailbox, &ctx) @@ -1066,7 +1153,7 @@ mod tests { index_uid: IndexUid::parse("test-index:0").unwrap(), source_id: "test-source".to_string(), }, - vec![(1, Position::Eof)], + vec![(1, Position::eof(11u64)), (2, Position::Beginning.as_eof())], ); let local_update = shard_positions_update_rx.recv().await.unwrap(); assert_eq!(local_update, expected_local_update); @@ -1112,7 +1199,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, shard_state: ShardState::Open as i32, - publish_position_inclusive: Some(11u64.into()), + publish_position_inclusive: Some(Position::offset(11u64)), publish_token: Some(publish_token.to_string()), }, Shard { @@ -1122,7 +1209,7 @@ mod tests { source_id: "test-source".to_string(), shard_id: 2, shard_state: ShardState::Closed as i32, - publish_position_inclusive: Some(Position::Eof), + publish_position_inclusive: Some(Position::eof(22u64)), publish_token: Some(publish_token.to_string()), }, ], @@ -1144,7 +1231,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.from_position_exclusive, Some(11u64.into())); + assert_eq!(request.from_position_exclusive(), Position::offset(11u64)); let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); Ok(service_stream) @@ -1163,13 +1250,13 @@ mod tests { assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); assert_eq!(subrequest.shard_id, 1); - assert_eq!(subrequest.to_position_inclusive, Some(11u64.into())); + assert_eq!(subrequest.to_position_inclusive(), Position::offset(11u64)); 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.to_position_inclusive, Some(Position::Eof)); + assert_eq!(subrequest.to_position_inclusive(), Position::eof(22u64)); let response = TruncateShardsResponse {}; Ok(response) @@ -1228,7 +1315,7 @@ mod tests { index_uid: IndexUid::parse("test-index:0").unwrap(), source_id: "test-source".to_string(), }, - vec![(1, 11u64.into()), (2, Position::Eof)], + vec![(1, Position::offset(11u64)), (2, Position::eof(22u64))], ); assert_eq!( local_shard_positions_update, @@ -1278,7 +1365,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: 11u64.into(), + current_position_inclusive: Position::offset(11u64), status: IndexingStatus::Active, }, ); @@ -1288,41 +1375,46 @@ mod tests { leader_id: "test-ingester-1".into(), follower_id_opt: None, partition_id: 2u64.into(), - current_position_inclusive: 22u64.into(), + current_position_inclusive: Position::offset(22u64), status: IndexingStatus::Active, }, ); - let fetch_response_tx = source.fetch_stream.fetch_response_tx(); + let fetch_message_tx = source.fetch_stream.fetch_message_tx(); - fetch_response_tx - .send(Ok(FetchResponseV2 { - index_uid: "test-index:0".into(), - source_id: "test-source".into(), - shard_id: 1, - mrecord_batch: Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc-112\0\0test-doc-113\0\x01"), - mrecord_lengths: vec![14, 14, 2], - }), - from_position_exclusive: Some(11u64.into()), - to_position_inclusive: Some(14u64.into()), - })) - .await - .unwrap(); + let fetch_payload = FetchPayload { + index_uid: "test-index:0".into(), + source_id: "test-source".into(), + shard_id: 1, + mrecord_batch: MRecordBatch::for_test([ + "\0\0test-doc-foo", + "\0\0test-doc-bar", + "\0\x01", + ]), + from_position_exclusive: Some(Position::offset(11u64)), + to_position_inclusive: Some(Position::offset(14u64)), + }; + let fetch_message = FetchMessage::new_payload(fetch_payload); + fetch_message_tx.send(Ok(fetch_message)).await.unwrap(); - fetch_response_tx - .send(Ok(FetchResponseV2 { - index_uid: "test-index:0".into(), - source_id: "test-source".into(), - shard_id: 2, - mrecord_batch: Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc-223\0\x02"), - mrecord_lengths: vec![14, 2], - }), - from_position_exclusive: Some(22u64.into()), - to_position_inclusive: Some(Position::Eof), - })) - .await - .unwrap(); + let fetch_payload = FetchPayload { + index_uid: "test-index:0".into(), + source_id: "test-source".into(), + shard_id: 2, + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-qux"]), + from_position_exclusive: Some(Position::offset(22u64)), + to_position_inclusive: Some(Position::offset(23u64)), + }; + let fetch_message = FetchMessage::new_payload(fetch_payload); + fetch_message_tx.send(Ok(fetch_message)).await.unwrap(); + + let fetch_eof = FetchEof { + index_uid: "test-index:0".into(), + source_id: "test-source".into(), + shard_id: 2, + eof_position: Some(Position::eof(23u64)), + }; + let fetch_message = FetchMessage::new_eof(fetch_eof); + fetch_message_tx.send(Ok(fetch_message)).await.unwrap(); source .emit_batches(&doc_processor_mailbox, &ctx) @@ -1333,9 +1425,9 @@ mod tests { .await .unwrap(); assert_eq!(doc_batch.docs.len(), 3); - assert_eq!(doc_batch.docs[0], "test-doc-112"); - assert_eq!(doc_batch.docs[1], "test-doc-113"); - assert_eq!(doc_batch.docs[2], "test-doc-223"); + assert_eq!(doc_batch.docs[0], "test-doc-foo"); + assert_eq!(doc_batch.docs[1], "test-doc-bar"); + assert_eq!(doc_batch.docs[2], "test-doc-qux"); assert!(doc_batch.force_commit); let partition_deltas = doc_batch @@ -1346,21 +1438,21 @@ mod tests { assert_eq!(partition_deltas.len(), 2); assert_eq!(partition_deltas[0].0, 1u64.into()); - assert_eq!(partition_deltas[0].1.from, Position::from(11u64)); - assert_eq!(partition_deltas[0].1.to, Position::from(14u64)); + assert_eq!(partition_deltas[0].1.from, Position::offset(11u64)); + assert_eq!(partition_deltas[0].1.to, Position::offset(14u64)); assert_eq!(partition_deltas[1].0, 2u64.into()); - assert_eq!(partition_deltas[1].1.from, Position::from(22u64)); - assert_eq!(partition_deltas[1].1.to, Position::Eof); + assert_eq!(partition_deltas[1].1.from, Position::offset(22u64)); + assert_eq!(partition_deltas[1].1.to, Position::eof(23u64)); source .emit_batches(&doc_processor_mailbox, &ctx) .await .unwrap(); let shard = source.assigned_shards.get(&2).unwrap(); - assert_eq!(shard.status, IndexingStatus::EofReached); + assert_eq!(shard.status, IndexingStatus::ReachedEof); - fetch_response_tx + fetch_message_tx .send(Err(FetchStreamError { index_uid: "test-index:0".into(), source_id: "test-source".into(), @@ -1377,20 +1469,16 @@ mod tests { let shard = source.assigned_shards.get(&1).unwrap(); assert_eq!(shard.status, IndexingStatus::Error); - fetch_response_tx - .send(Ok(FetchResponseV2 { - index_uid: "test-index:0".into(), - source_id: "test-source".into(), - shard_id: 1, - mrecord_batch: Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc-114"), - mrecord_lengths: vec![14], - }), - from_position_exclusive: Some(14u64.into()), - to_position_inclusive: Some(15u64.into()), - })) - .await - .unwrap(); + let fetch_payload = FetchPayload { + index_uid: "test-index:0".into(), + source_id: "test-source".into(), + shard_id: 1, + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-baz"]), + from_position_exclusive: Some(Position::offset(14u64)), + to_position_inclusive: Some(Position::offset(15u64)), + }; + let fetch_message = FetchMessage::new_payload(fetch_payload); + fetch_message_tx.send(Ok(fetch_message)).await.unwrap(); source .emit_batches(&doc_processor_mailbox, &ctx) @@ -1423,15 +1511,21 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 1); - assert_eq!(subrequest_0.to_position_inclusive, Some(11u64.into())); + assert_eq!( + subrequest_0.to_position_inclusive(), + Position::offset(11u64) + ); let subrequest_1 = &request.subrequests[1]; assert_eq!(subrequest_1.shard_id, 2); - assert_eq!(subrequest_1.to_position_inclusive, Some(22u64.into())); + assert_eq!( + subrequest_1.to_position_inclusive(), + Position::offset(22u64) + ); let subrequest_2 = &request.subrequests[2]; assert_eq!(subrequest_2.shard_id, 3); - assert_eq!(subrequest_2.to_position_inclusive, Some(Position::Eof)); + assert_eq!(subrequest_2.to_position_inclusive(), Position::eof(33u64)); Ok(TruncateShardsResponse {}) }); @@ -1448,11 +1542,14 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 2); - assert_eq!(subrequest_0.to_position_inclusive, Some(22u64.into())); + assert_eq!( + subrequest_0.to_position_inclusive(), + Position::offset(22u64) + ); let subrequest_1 = &request.subrequests[1]; assert_eq!(subrequest_1.shard_id, 3); - assert_eq!(subrequest_1.to_position_inclusive, Some(Position::Eof)); + assert_eq!(subrequest_1.to_position_inclusive(), Position::eof(33u64)); Ok(TruncateShardsResponse {}) }); @@ -1469,7 +1566,10 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 4); - assert_eq!(subrequest_0.to_position_inclusive, Some(44u64.into())); + assert_eq!( + subrequest_0.to_position_inclusive(), + Position::offset(44u64) + ); Ok(TruncateShardsResponse {}) }); @@ -1512,7 +1612,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: 11u64.into(), + current_position_inclusive: Position::offset(11u64), status: IndexingStatus::Active, }, ); @@ -1522,7 +1622,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: Some("test-ingester-1".into()), partition_id: 2u64.into(), - current_position_inclusive: 22u64.into(), + current_position_inclusive: Position::offset(22u64), status: IndexingStatus::Active, }, ); @@ -1532,7 +1632,7 @@ mod tests { leader_id: "test-ingester-1".into(), follower_id_opt: Some("test-ingester-0".into()), partition_id: 3u64.into(), - current_position_inclusive: 33u64.into(), + current_position_inclusive: Position::offset(33u64), status: IndexingStatus::Active, }, ); @@ -1542,7 +1642,7 @@ mod tests { leader_id: "test-ingester-2".into(), follower_id_opt: Some("test-ingester-3".into()), partition_id: 4u64.into(), - current_position_inclusive: 44u64.into(), + current_position_inclusive: Position::offset(44u64), status: IndexingStatus::Active, }, ); @@ -1551,19 +1651,19 @@ mod tests { AssignedShard { leader_id: "test-ingester-2".into(), follower_id_opt: Some("test-ingester-3".into()), - partition_id: 4u64.into(), + partition_id: 5u64.into(), current_position_inclusive: Position::Beginning, status: IndexingStatus::Active, }, ); let checkpoint = SourceCheckpoint::from_iter(vec![ - (1u64.into(), 11u64.into()), - (2u64.into(), 22u64.into()), - (3u64.into(), Position::Eof), - (4u64.into(), 44u64.into()), + (1u64.into(), Position::offset(11u64)), + (2u64.into(), Position::offset(22u64)), + (3u64.into(), Position::eof(33u64)), + (4u64.into(), Position::offset(44u64)), (5u64.into(), Position::Beginning), - (6u64.into(), 66u64.into()), + (6u64.into(), Position::offset(66u64)), ]); source.suggest_truncate(checkpoint, &ctx).await.unwrap(); @@ -1574,12 +1674,12 @@ mod tests { source_id: "test-source".to_string(), }, vec![ - (1u64, 11u64.into()), - (2u64, 22u64.into()), - (3u64, Position::Eof), - (4u64, 44u64.into()), + (1u64, Position::offset(11u64)), + (2u64, Position::offset(22u64)), + (3u64, Position::eof(33u64)), + (4u64, Position::offset(44u64)), (5u64, Position::Beginning), - (6u64, 66u64.into()), + (6u64, Position::offset(66u64)), ], ); assert_eq!(local_shards_update, expected_local_shards_update); diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index ab1d14144d8..c00640278bd 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -182,9 +182,9 @@ impl Source for IngestApiSource { partition_id, self.counters .previous_offset - .map(Position::from) - .unwrap_or(Position::Beginning), - Position::from(current_offset), + .map(Position::offset) + .unwrap_or_default(), + Position::offset(current_offset), ) .map_err(anyhow::Error::from)?; @@ -408,8 +408,8 @@ mod tests { let partition_id: PartitionId = ingest_api_service.ask(GetPartitionId).await?.into(); let checkpoint_delta = SourceCheckpointDelta::from_partition_delta( partition_id.clone(), - Position::from(0u64), - Position::from(1200u64), + Position::offset(0u64), + Position::offset(1200u64), ) .unwrap(); checkpoint.try_apply_delta(checkpoint_delta).unwrap(); @@ -686,7 +686,7 @@ mod tests { let partition_id = ingest_api_service.ask(GetPartitionId).await?.into(); let mut source_checkpoint = SourceCheckpoint::default(); - source_checkpoint.add_partition(partition_id, Position::from(10u64)); + source_checkpoint.add_partition(partition_id, Position::offset(10u64)); let ingest_api_source = IngestApiSource::try_new(ctx, source_checkpoint).await?; let ingest_api_source_actor = SourceActor { source: Box::new(ingest_api_source), diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index 3bc1259289a..47c3278e03a 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -323,7 +323,7 @@ impl KafkaSource { ) })? .clone(); - let current_position = Position::from(offset); + let current_position = Position::offset(offset); let previous_position = self .state .current_positions @@ -384,7 +384,7 @@ impl KafkaSource { .expect("Kafka offset should be stored as i64"); Offset::Offset(offset + 1) } - Position::Eof => { + Position::Eof(_) => { panic!("position of a Kafka partition should never be EOF") } }; @@ -542,11 +542,11 @@ impl Source for KafkaSource { fn observable_state(&self) -> JsonValue { let assigned_partitions: Vec<&i32> = self.state.assigned_partitions.keys().sorted().collect(); - let current_positions: Vec<(&i32, &str)> = self + let current_positions: Vec<(&i32, &Position)> = self .state .current_positions .iter() - .map(|(partition, position)| (partition, position.as_str())) + .map(|(partition, position)| (partition, position)) .sorted() .collect(); json!({ @@ -638,7 +638,7 @@ fn previous_position_for_offset(offset: i64) -> Position { if offset == 0 { Position::Beginning } else { - Position::from(offset - 1) + Position::offset(offset - 1) } } @@ -957,18 +957,18 @@ mod kafka_broker_tests { metastore.stage_splits(stage_splits_request).await.unwrap(); let mut source_delta = SourceCheckpointDelta::default(); - for (partition_id, from_position, to_position) in partition_deltas { + for (partition_id, from_position, to_position) in partition_deltas.iter().copied() { source_delta .record_partition_delta( - (*partition_id).into(), + partition_id.into(), { - if *from_position < 0 { + if from_position < 0 { Position::Beginning } else { - (*from_position).into() + Position::offset(from_position as u64) } }, - (*to_position).into(), + Position::offset(to_position as u64), ) .unwrap(); } @@ -1044,7 +1044,7 @@ mod kafka_broker_tests { assert_eq!(batch.num_bytes, 0); assert_eq!( kafka_source.state.current_positions.get(&1).unwrap(), - &Position::from(0u64) + &Position::offset(0u64) ); assert_eq!(kafka_source.state.num_bytes_processed, 7); assert_eq!(kafka_source.state.num_messages_processed, 1); @@ -1066,7 +1066,7 @@ mod kafka_broker_tests { assert_eq!(batch.num_bytes, 8); assert_eq!( kafka_source.state.current_positions.get(&1).unwrap(), - &Position::from(1u64) + &Position::offset(1u64) ); assert_eq!(kafka_source.state.num_bytes_processed, 15); assert_eq!(kafka_source.state.num_messages_processed, 2); @@ -1088,7 +1088,7 @@ mod kafka_broker_tests { assert_eq!(batch.num_bytes, 16); assert_eq!( kafka_source.state.current_positions.get(&2).unwrap(), - &Position::from(42u64) + &Position::offset(42u64) ); assert_eq!(kafka_source.state.num_bytes_processed, 23); assert_eq!(kafka_source.state.num_messages_processed, 3); @@ -1096,10 +1096,14 @@ mod kafka_broker_tests { let mut expected_checkpoint_delta = SourceCheckpointDelta::default(); expected_checkpoint_delta - .record_partition_delta(partition_id_1, Position::Beginning, Position::from(1u64)) + .record_partition_delta(partition_id_1, Position::Beginning, Position::offset(1u64)) .unwrap(); expected_checkpoint_delta - .record_partition_delta(partition_id_2, Position::from(41u64), Position::from(42u64)) + .record_partition_delta( + partition_id_2, + Position::offset(41u64), + Position::offset(42u64), + ) .unwrap(); assert_eq!(batch.checkpoint_delta, expected_checkpoint_delta); @@ -1166,7 +1170,7 @@ mod kafka_broker_tests { kafka_source.state.assigned_partitions, expected_assigned_partitions ); - let expected_current_positions = HashMap::from_iter([(2, Position::from(42u64))]); + let expected_current_positions = HashMap::from_iter([(2, Position::offset(42u64))]); assert_eq!( kafka_source.state.current_positions, expected_current_positions @@ -1322,7 +1326,9 @@ mod kafka_broker_tests { let checkpoint: SourceCheckpoint = [(0u64, 1u64), (1u64, 2u64)] .into_iter() - .map(|(partition_id, offset)| (PartitionId::from(partition_id), Position::from(offset))) + .map(|(partition_id, offset)| { + (PartitionId::from(partition_id), Position::offset(offset)) + }) .collect(); kafka_source.truncate(checkpoint).unwrap(); @@ -1465,7 +1471,7 @@ mod kafka_broker_tests { expected_checkpoint_delta.record_partition_delta( PartitionId::from(partition), Position::Beginning, - Position::from(2u64), + Position::offset(2u64), )?; } assert_eq!(batch.checkpoint_delta, expected_checkpoint_delta); @@ -1527,13 +1533,13 @@ mod kafka_broker_tests { let mut expected_checkpoint_delta = SourceCheckpointDelta::default(); expected_checkpoint_delta.record_partition_delta( PartitionId::from(0u64), - Position::from(0u64), - Position::from(2u64), + Position::offset(0u64), + Position::offset(2u64), )?; expected_checkpoint_delta.record_partition_delta( PartitionId::from(2u64), Position::Beginning, - Position::from(2u64), + Position::offset(2u64), )?; assert_eq!(batch.checkpoint_delta, expected_checkpoint_delta,); diff --git a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs index 4064ccc4407..bc050154e14 100644 --- a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs +++ b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs @@ -153,7 +153,7 @@ impl KinesisSource { let from_sequence_number_exclusive = match &position { Position::Beginning => None, Position::Offset(offset) => Some(offset.to_string()), - Position::Eof => panic!("position of a Kinesis shard should never be EOF"), + Position::Eof(_) => panic!("position of a Kinesis shard should never be EOF"), }; let shard_consumer = ShardConsumer::new( self.stream_name.clone(), @@ -331,13 +331,11 @@ impl Source for KinesisSource { } fn observable_state(&self) -> JsonValue { - let shard_consumer_positions: Vec<(&ShardId, &str)> = self + let shard_consumer_positions: Vec<(&ShardId, &Position)> = self .state .shard_consumers .iter() - .map(|(shard_id, shard_consumer_state)| { - (shard_id, shard_consumer_state.position.as_str()) - }) + .map(|(shard_id, shard_consumer_state)| (shard_id, &shard_consumer_state.position)) .sorted() .collect(); json!({ diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index cb10092c499..0cb91b4605b 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -337,7 +337,7 @@ fn msg_id_to_position(msg: &MessageIdData) -> Position { // in order to re-construct the message ID in order to send back to pulsar. // The ledger_id, entry_id and the batch_index form a unique composite key which will // prevent the remaining parts of the ID from interfering with the sorting. - let id_str = format!( + let position_str = format!( "{:0>20},{:0>20},{},{},{}", msg.ledger_id, msg.entry_id, @@ -356,12 +356,14 @@ fn msg_id_to_position(msg: &MessageIdData) -> Position { .unwrap_or_default(), ); - Position::from(id_str) + Position::from(position_str) } -fn msg_id_from_position(pos: &Position) -> Option { - let id_str = pos.as_str(); - let mut parts = id_str.split(','); +fn msg_id_from_position(position: &Position) -> Option { + let Position::Offset(offset) = position else { + return None; + }; + let mut parts = offset.as_str().split(','); let ledger_id = parts.next()?.parse::().ok()?; let entry_id = parts.next()?.parse::().ok()?; @@ -460,7 +462,7 @@ mod pulsar_broker_tests { ($($partition:expr => $position:expr $(,)?)*) => {{ let mut positions = BTreeMap::new(); $( - positions.insert(PartitionId::from($partition), Position::from($position)); + positions.insert(PartitionId::from($partition), Position::offset($position)); )* positions }}; @@ -473,7 +475,7 @@ mod pulsar_broker_tests { checkpoint.record_partition_delta( PartitionId::from($partition), Position::Beginning, - Position::from($position), + $position, ).unwrap(); )* checkpoint @@ -756,7 +758,7 @@ mod pulsar_broker_tests { let position = msg_id_to_position(&populated_id); assert_eq!( - position.as_str(), + position.to_string(), format!("{:0>20},{:0>20},{:010},,{:010}", 1, 134, 3, 6) ); let retrieved_id = msg_id_from_position(&position) @@ -777,7 +779,7 @@ mod pulsar_broker_tests { let position = msg_id_to_position(&partitioned_id); assert_eq!( - position.as_str(), + position.to_string(), format!("{:0>20},{:0>20},{:010},{:010},{:010}", 1, 134, 3, 5, 6) ); let retrieved_id = msg_id_from_position(&position) @@ -798,7 +800,7 @@ mod pulsar_broker_tests { let position = msg_id_to_position(&sparse_id); assert_eq!( - position.as_str(), + position.to_string(), format!("{:0>20},{:0>20},,,{:010}", 1, 4, 0) ); let retrieved_id = msg_id_from_position(&position) @@ -844,7 +846,7 @@ mod pulsar_broker_tests { assert_eq!(batch.num_bytes, 0); assert!(batch.docs.is_empty()); - let position = Position::from(1u64); // Used for testing simplicity. + let position = Position::offset(1u64); // Used for testing simplicity. let mut batch = BatchBuilder::default(); let doc = Bytes::from_static(b"some-demo-data"); pulsar_source @@ -861,7 +863,7 @@ mod pulsar_broker_tests { assert_eq!(batch.num_bytes, 14); assert_eq!(batch.docs.len(), 1); - let position = Position::from(4u64); // Used for testing simplicity. + let position = Position::offset(4u64); // Used for testing simplicity. let mut batch = BatchBuilder::default(); let doc = Bytes::from_static(b"some-demo-data-2"); pulsar_source @@ -881,8 +883,8 @@ mod pulsar_broker_tests { expected_checkpoint_delta .record_partition_delta( PartitionId::from(topic.as_str()), - Position::from(1u64), - Position::from(4u64), + Position::offset(1u64), + Position::offset(4u64), ) .unwrap(); assert_eq!(batch.checkpoint_delta, expected_checkpoint_delta); diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index 79271a53712..02114e7be47 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -82,7 +82,7 @@ fn position_from_offset(offset: usize) -> Position { if offset == 0 { return Position::Beginning; } - Position::from(offset - 1) + Position::offset(offset - 1) } #[async_trait] diff --git a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs index 90935cdc9b1..dde54fd408a 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/broadcast.rs @@ -194,9 +194,9 @@ impl BroadcastLocalShardsTask { source_id, }; // Shard ingestion rate in MiB/s. - let ingestion_rate_u64 = - rate_meter.harvest().rescale(Duration::from_secs(1)).work() / ONE_MIB.as_u64(); - let ingestion_rate = RateMibPerSec(ingestion_rate_u64 as u16); + let ingestion_rate_per_sec = rate_meter.harvest().rescale(Duration::from_secs(1)); + let ingestion_rate_mib_per_sec_u64 = ingestion_rate_per_sec.work() / ONE_MIB.as_u64(); + let ingestion_rate = RateMibPerSec(ingestion_rate_mib_per_sec_u64 as u16); let shard_info = ShardInfo { shard_id, @@ -471,7 +471,8 @@ mod tests { let mut state_guard = state.write().await; let queue_id_01 = queue_id("test-index:0", "test-source", 1); - let shard = IngesterShard::new_solo(ShardState::Open, Position::Beginning, Position::Eof); + let shard = + IngesterShard::new_solo(ShardState::Open, Position::Beginning, Position::Beginning); state_guard.shards.insert(queue_id_01.clone(), shard); let rate_limiter = RateLimiter::from_settings(RateLimiterSettings::default()); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index ce2113c1592..bc52dbf4404 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -27,7 +27,9 @@ use bytes::{BufMut, BytesMut}; use futures::StreamExt; use quickwit_common::retry::RetryParams; use quickwit_common::ServiceStream; -use quickwit_proto::ingest::ingester::{FetchResponseV2, IngesterService, OpenFetchStreamRequest}; +use quickwit_proto::ingest::ingester::{ + fetch_message, FetchEof, FetchMessage, FetchPayload, IngesterService, OpenFetchStreamRequest, +}; use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, MRecordBatch}; use quickwit_proto::types::{queue_id, IndexUid, NodeId, Position, QueueId, ShardId, SourceId}; use tokio::sync::{mpsc, watch, RwLock}; @@ -35,11 +37,11 @@ use tokio::task::JoinHandle; use tracing::{debug, error, warn}; use super::ingester::IngesterState; -use crate::ingest_v2::mrecord::is_eof_mrecord; +use super::models::ShardStatus; use crate::{ClientId, IngesterPool}; /// A fetch stream task is responsible for waiting and pushing new records written to a shard's -/// record log into a channel named `fetch_response_tx`. +/// record log into a channel named `fetch_message_tx`. pub(super) struct FetchStreamTask { /// Uniquely identifies the consumer of the fetch task for logging and debugging purposes. client_id: ClientId, @@ -50,10 +52,10 @@ pub(super) struct FetchStreamTask { /// The position of the next record fetched. from_position_inclusive: u64, state: Arc>, - fetch_response_tx: mpsc::Sender>, + fetch_message_tx: mpsc::Sender>, /// This channel notifies the fetch task when new records are available. This way the fetch /// task does not need to grab the lock and poll the mrecordlog queue unnecessarily. - new_records_rx: watch::Receiver<()>, + shard_status_rx: watch::Receiver, batch_num_bytes: usize, } @@ -74,18 +76,15 @@ impl FetchStreamTask { pub fn spawn( open_fetch_stream_request: OpenFetchStreamRequest, state: Arc>, - new_records_rx: watch::Receiver<()>, + shard_status_rx: watch::Receiver, batch_num_bytes: usize, - ) -> ( - ServiceStream>, - JoinHandle<()>, - ) { + ) -> (ServiceStream>, JoinHandle<()>) { let from_position_inclusive = open_fetch_stream_request .from_position_exclusive() .as_u64() .map(|offset| offset + 1) .unwrap_or_default(); - let (fetch_response_tx, fetch_stream) = ServiceStream::new_bounded(3); + let (fetch_message_tx, fetch_stream) = ServiceStream::new_bounded(3); let mut fetch_task = Self { queue_id: open_fetch_stream_request.queue_id(), client_id: open_fetch_stream_request.client_id, @@ -94,8 +93,8 @@ impl FetchStreamTask { shard_id: open_fetch_stream_request.shard_id, from_position_inclusive, state, - fetch_response_tx, - new_records_rx, + fetch_message_tx, + shard_status_rx, batch_num_bytes, }; let future = async move { fetch_task.run().await }; @@ -115,14 +114,15 @@ impl FetchStreamTask { "spawning fetch task" ); let mut has_drained_queue = false; - let mut has_reached_eof = false; + let mut to_position_inclusive = Position::Beginning; - while !has_reached_eof { - if has_drained_queue && self.new_records_rx.changed().await.is_err() { + loop { + if has_drained_queue && self.shard_status_rx.changed().await.is_err() { // The shard was dropped. break; } has_drained_queue = true; + let mut mrecord_buffer = BytesMut::with_capacity(self.batch_num_bytes); let mut mrecord_lengths = Vec::new(); @@ -146,59 +146,67 @@ impl FetchStreamTask { // Drop the lock while we send the message. drop(state_guard); - if mrecord_buffer.is_empty() { - continue; + if !mrecord_lengths.is_empty() { + let from_position_exclusive = if self.from_position_inclusive == 0 { + Position::Beginning + } else { + Position::offset(self.from_position_inclusive - 1) + }; + self.from_position_inclusive += mrecord_lengths.len() as u64; + + to_position_inclusive = Position::offset(self.from_position_inclusive - 1); + + let mrecord_batch = MRecordBatch { + mrecord_buffer: mrecord_buffer.freeze(), + mrecord_lengths, + }; + let fetch_payload = FetchPayload { + index_uid: self.index_uid.clone().into(), + source_id: self.source_id.clone(), + shard_id: self.shard_id, + mrecord_batch: Some(mrecord_batch), + from_position_exclusive: Some(from_position_exclusive), + to_position_inclusive: Some(to_position_inclusive.clone()), + }; + let fetch_message = FetchMessage::new_payload(fetch_payload); + + if self.fetch_message_tx.send(Ok(fetch_message)).await.is_err() { + // The consumer was dropped. + return; + } } - let from_position_exclusive = if self.from_position_inclusive == 0 { - Position::Beginning - } else { - Position::from(self.from_position_inclusive - 1) - }; - self.from_position_inclusive += mrecord_lengths.len() as u64; - - let last_mrecord_len = *mrecord_lengths - .last() - .expect("`mrecord_lengths` should not be empty") - as usize; - let last_mrecord = &mrecord_buffer[mrecord_buffer.len() - last_mrecord_len..]; - - let to_position_inclusive = if is_eof_mrecord(last_mrecord) { - debug!( - client_id=%self.client_id, - index_uid=%self.index_uid, - source_id=%self.source_id, - shard_id=%self.shard_id, - to_position_inclusive=%self.from_position_inclusive - 1, - "fetch stream reached end of shard" - ); - has_reached_eof = true; - Position::Eof - } else { - Position::from(self.from_position_inclusive - 1) - }; - let mrecord_batch = MRecordBatch { - mrecord_buffer: mrecord_buffer.freeze(), - mrecord_lengths, - }; - let fetch_response = FetchResponseV2 { - index_uid: self.index_uid.clone().into(), - source_id: self.source_id.clone(), - shard_id: self.shard_id, - mrecord_batch: Some(mrecord_batch), - from_position_exclusive: Some(from_position_exclusive), - to_position_inclusive: Some(to_position_inclusive), - }; - if self - .fetch_response_tx - .send(Ok(fetch_response)) - .await - .is_err() - { - // The consumer was dropped. - break; + if has_drained_queue { + let has_reached_eof = { + let shard_status = self.shard_status_rx.borrow(); + let shard_state = &shard_status.0; + let replication_position = &shard_status.1; + shard_state.is_closed() && to_position_inclusive >= *replication_position + }; + if has_reached_eof { + debug!( + client_id=%self.client_id, + index_uid=%self.index_uid, + source_id=%self.source_id, + shard_id=%self.shard_id, + to_position_inclusive=%self.from_position_inclusive - 1, + "fetch stream reached end of shard" + ); + let eof_position = to_position_inclusive.as_eof(); + + let fetch_eof = FetchEof { + index_uid: self.index_uid.clone().into(), + source_id: self.source_id.clone(), + shard_id: self.shard_id, + eof_position: Some(eof_position), + }; + let fetch_message = FetchMessage::new_eof(fetch_eof); + + let _ = self.fetch_message_tx.send(Ok(fetch_message)).await; + return; + } } } - if !has_reached_eof { + if !to_position_inclusive.is_eof() { error!( client_id=%self.client_id, index_uid=%self.index_uid, @@ -207,7 +215,7 @@ impl FetchStreamTask { "fetch stream ended before reaching end of shard" ); let _ = self - .fetch_response_tx + .fetch_message_tx .send(Err(IngestV2Error::Internal( "fetch stream ended before reaching end of shard".to_string(), ))) @@ -232,8 +240,8 @@ pub struct MultiFetchStream { ingester_pool: IngesterPool, retry_params: RetryParams, fetch_task_handles: HashMap>, - fetch_response_rx: mpsc::Receiver>, - fetch_response_tx: mpsc::Sender>, + fetch_message_rx: mpsc::Receiver>, + fetch_message_tx: mpsc::Sender>, } impl MultiFetchStream { @@ -243,21 +251,21 @@ impl MultiFetchStream { ingester_pool: IngesterPool, retry_params: RetryParams, ) -> Self { - let (fetch_response_tx, fetch_response_rx) = mpsc::channel(3); + let (fetch_message_tx, fetch_message_rx) = mpsc::channel(3); Self { self_node_id, client_id, ingester_pool, retry_params, fetch_task_handles: HashMap::new(), - fetch_response_rx, - fetch_response_tx, + fetch_message_rx, + fetch_message_tx, } } #[cfg(any(test, feature = "testsuite"))] - pub fn fetch_response_tx(&self) -> mpsc::Sender> { - self.fetch_response_tx.clone() + pub fn fetch_message_tx(&self) -> mpsc::Sender> { + self.fetch_message_tx.clone() } /// Subscribes to a shard and fails over to the replica if an error occurs. @@ -297,7 +305,7 @@ impl MultiFetchStream { ingester_ids, self.ingester_pool.clone(), self.retry_params, - self.fetch_response_tx.clone(), + self.fetch_message_tx.clone(), ); let fetch_task_handle = tokio::spawn(fetch_stream_future); self.fetch_task_handles.insert(queue_id, fetch_task_handle); @@ -323,10 +331,10 @@ impl MultiFetchStream { /// # Cancel safety /// /// This method is cancel safe. - pub async fn next(&mut self) -> Result { + pub async fn next(&mut self) -> Result { // Because we always hold a sender and never call `close()` on the receiver, the channel is // always open. - self.fetch_response_rx + self.fetch_message_rx .recv() .await .expect("the channel should be open") @@ -340,9 +348,9 @@ impl MultiFetchStream { for (_queue_id, fetch_stream_handle) in self.fetch_task_handles.drain() { fetch_stream_handle.abort(); } - let (fetch_response_tx, fetch_response_rx) = mpsc::channel(3); - self.fetch_response_tx = fetch_response_tx; - self.fetch_response_rx = fetch_response_rx; + let (fetch_message_tx, fetch_message_rx) = mpsc::channel(3); + self.fetch_message_tx = fetch_message_tx; + self.fetch_message_rx = fetch_message_rx; } } @@ -385,7 +393,7 @@ async fn retrying_fetch_stream( ingester_ids: Vec, ingester_pool: IngesterPool, retry_params: RetryParams, - fetch_response_tx: mpsc::Sender>, + fetch_message_tx: mpsc::Sender>, ) { for num_attempts in 1..=retry_params.max_attempts { fault_tolerant_fetch_stream( @@ -396,11 +404,11 @@ async fn retrying_fetch_stream( &mut from_position_exclusive, &ingester_ids, ingester_pool.clone(), - fetch_response_tx.clone(), + fetch_message_tx.clone(), ) .await; - if from_position_exclusive == Position::Eof { + if from_position_exclusive.is_eof() { break; } let delay = retry_params.compute_delay(num_attempts); @@ -419,7 +427,7 @@ async fn fault_tolerant_fetch_stream( from_position_exclusive: &mut Position, ingester_ids: &[NodeId], ingester_pool: IngesterPool, - fetch_response_tx: mpsc::Sender>, + fetch_message_tx: mpsc::Sender>, ) { // TODO: We can probably simplify this code by breaking it into smaller functions. 'outer: for (ingester_idx, ingester_id) in ingester_ids.iter().enumerate() { @@ -453,7 +461,7 @@ async fn fault_tolerant_fetch_stream( shard_id, ingest_error, }; - let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + let _ = fetch_message_tx.send(Err(fetch_stream_error)).await; return; } continue; @@ -492,28 +500,39 @@ async fn fault_tolerant_fetch_stream( shard_id, ingest_error, }; - let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + let _ = fetch_message_tx.send(Err(fetch_stream_error)).await; return; } continue; } }; - while let Some(fetch_response_result) = fetch_stream.next().await { - match fetch_response_result { - Ok(fetch_response) => { - let to_position_inclusive = fetch_response.to_position_inclusive(); - - if fetch_response_tx.send(Ok(fetch_response)).await.is_err() { - // The stream was dropped. - return; + while let Some(fetch_message_result) = fetch_stream.next().await { + 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(); + + if fetch_message_tx.send(Ok(fetch_message)).await.is_err() { + // The consumer was dropped. + return; + } + *from_position_exclusive = to_position_inclusive; } - *from_position_exclusive = to_position_inclusive; + Some(fetch_message::Message::Eof(fetch_eof)) => { + let eof_position = fetch_eof.eof_position(); - if *from_position_exclusive == Position::Eof { - // The stream has reached the end of the shard. + // We ignore the send error if the consumer was dropped because we're going + // to return anyway. + let _ = fetch_message_tx.send(Ok(fetch_message)).await; + + *from_position_exclusive = eof_position; return; } - } + None => { + warn!("received empty fetch message"); + continue; + } + }, Err(ingest_error) => { if let Some(failover_ingester_id) = failover_ingester_id_opt { warn!( @@ -539,7 +558,7 @@ async fn fault_tolerant_fetch_stream( shard_id, ingest_error, }; - let _ = fetch_response_tx.send(Err(fetch_stream_error)).await; + let _ = fetch_message_tx.send(Err(fetch_stream_error)).await; return; } continue 'outer; @@ -550,7 +569,7 @@ async fn fault_tolerant_fetch_stream( } #[cfg(test)] -mod tests { +pub(super) mod tests { use std::time::Duration; use bytes::Bytes; @@ -558,12 +577,27 @@ mod tests { use quickwit_proto::ingest::ingester::{ IngesterServiceClient, IngesterStatus, ObservationMessage, }; + use quickwit_proto::ingest::ShardState; use quickwit_proto::types::queue_id; use tokio::time::timeout; use super::*; use crate::MRecord; + pub fn into_fetch_payload(fetch_message: FetchMessage) -> FetchPayload { + match fetch_message.message.unwrap() { + fetch_message::Message::Payload(fetch_payload) => fetch_payload, + other => panic!("expected fetch payload, got `{other:?}`"), + } + } + + pub fn into_fetch_eof(fetch_message: FetchMessage) -> FetchEof { + match fetch_message.message.unwrap() { + fetch_message::Message::Eof(fetch_eof) => fetch_eof, + other => panic!("expected fetch EOF, got `{other:?}`"), + } + } + #[tokio::test] async fn test_fetch_task_happy_path() { let tempdir = tempfile::tempdir().unwrap(); @@ -588,11 +622,11 @@ mod tests { status: IngesterStatus::Ready, observation_tx, })); - let (new_records_tx, new_records_rx) = watch::channel(()); + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); let (mut fetch_stream, fetch_task_handle) = FetchStreamTask::spawn( open_fetch_stream_request, state.clone(), - new_records_rx, + shard_status_rx, 1024, ); let queue_id = queue_id(&index_uid, &source_id, 1); @@ -611,21 +645,23 @@ mod tests { .unwrap(); drop(state_guard); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!(fetch_response.index_uid, "test-index:0"); - assert_eq!(fetch_response.source_id, "test-source"); - assert_eq!(fetch_response.shard_id, 1); + let fetch_payload = into_fetch_payload(fetch_message); + + 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.from_position_exclusive(), Position::Beginning); assert_eq!( - fetch_response.from_position_exclusive(), - Position::Beginning + fetch_payload.to_position_inclusive(), + Position::offset(0u64) ); - assert_eq!(fetch_response.to_position_inclusive(), 0u64); assert_eq!( - fetch_response + fetch_payload .mrecord_batch .as_ref() .unwrap() @@ -633,14 +669,22 @@ mod tests { [14] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, "\0\0test-doc-foo" ); + timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap_err(); + + // Trigger a spurious notification. + let shard_status = (ShardState::Open, Position::offset(0u64)); + shard_status_tx.send(shard_status).unwrap(); + + timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap_err(); + let mut state_guard = state.write().await; state_guard @@ -650,27 +694,26 @@ mod tests { .unwrap(); drop(state_guard); - timeout(Duration::from_millis(50), fetch_stream.next()) - .await - .unwrap_err(); - - new_records_tx.send(()).unwrap(); + let shard_status = (ShardState::Open, Position::offset(1u64)); + shard_status_tx.send(shard_status.clone()).unwrap(); - // Trigger a spurious notification. - new_records_tx.send(()).unwrap(); - - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.from_position_exclusive(), + Position::offset(0u64) ); - assert_eq!(fetch_response.to_position_inclusive(), 1u64); assert_eq!( - fetch_response + fetch_payload.to_position_inclusive(), + Position::offset(1u64) + ); + assert_eq!( + fetch_payload .mrecord_batch .as_ref() .unwrap() @@ -678,11 +721,7 @@ mod tests { [14] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, "\0\0test-doc-bar" ); @@ -691,7 +730,6 @@ mod tests { let mrecords = [ MRecord::new_doc("test-doc-baz").encode(), MRecord::new_doc("test-doc-qux").encode(), - MRecord::Eof.encode(), ] .into_iter(); @@ -702,34 +740,116 @@ mod tests { .unwrap(); drop(state_guard); - new_records_tx.send(()).unwrap(); + let shard_status = (ShardState::Open, Position::offset(3u64)); + shard_status_tx.send(shard_status).unwrap(); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + + assert_eq!( + fetch_payload.from_position_exclusive(), + Position::offset(1u64) + ); assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(1u64) + fetch_payload.to_position_inclusive(), + Position::offset(3u64) ); - assert_eq!(fetch_response.to_position_inclusive(), Position::Eof); assert_eq!( - fetch_response + fetch_payload .mrecord_batch .as_ref() .unwrap() .mrecord_lengths, - [14, 14, 2] + [14, 14] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, - "\0\0test-doc-baz\0\0test-doc-qux\0\x02" + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, + "\0\0test-doc-baz\0\0test-doc-qux" + ); + + let shard_status = (ShardState::Closed, Position::offset(3u64)); + shard_status_tx.send(shard_status).unwrap(); + + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap() + .unwrap() + .unwrap(); + let fetch_eof = into_fetch_eof(fetch_message); + + 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.eof_position, Some(Position::eof(3u64))); + + fetch_task_handle.await.unwrap(); + } + + #[tokio::test] + async fn test_fetch_task_eof_at_beginning() { + let tempdir = tempfile::tempdir().unwrap(); + let mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); + let client_id = "test-client".to_string(); + let index_uid = "test-index:0".to_string(); + let source_id = "test-source".to_string(); + let open_fetch_stream_request = OpenFetchStreamRequest { + client_id: client_id.clone(), + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: 1, + from_position_exclusive: None, + }; + let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); + let state = Arc::new(RwLock::new(IngesterState { + mrecordlog, + shards: HashMap::new(), + rate_trackers: HashMap::new(), + replication_streams: HashMap::new(), + replication_tasks: HashMap::new(), + status: IngesterStatus::Ready, + observation_tx, + })); + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let (mut fetch_stream, fetch_task_handle) = FetchStreamTask::spawn( + open_fetch_stream_request, + state.clone(), + shard_status_rx, + 1024, ); + let queue_id = queue_id(&index_uid, &source_id, 1); + + let mut state_guard = state.write().await; + + state_guard + .mrecordlog + .create_queue(&queue_id) + .await + .unwrap(); + drop(state_guard); + + timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap_err(); + + let shard_status = (ShardState::Closed, Position::Beginning); + shard_status_tx.send(shard_status).unwrap(); + + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap() + .unwrap() + .unwrap(); + let fetch_eof = into_fetch_eof(fetch_message); + + 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.eof_position, Some(Position::Beginning.as_eof())); + fetch_task_handle.await.unwrap(); } @@ -745,7 +865,7 @@ mod tests { index_uid: index_uid.clone(), source_id: source_id.clone(), shard_id: 1, - from_position_exclusive: Some(Position::from(0u64)), + from_position_exclusive: Some(Position::offset(0u64)), }; let (observation_tx, _observation_rx) = watch::channel(Ok(ObservationMessage::default())); let state = Arc::new(RwLock::new(IngesterState { @@ -757,11 +877,11 @@ mod tests { status: IngesterStatus::Ready, observation_tx, })); - let (new_records_tx, new_records_rx) = watch::channel(()); + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); let (mut fetch_stream, _fetch_task_handle) = FetchStreamTask::spawn( open_fetch_stream_request, state.clone(), - new_records_rx, + shard_status_rx, 1024, ); let queue_id = queue_id(&index_uid, &source_id, 1); @@ -775,8 +895,6 @@ mod tests { .unwrap(); drop(state_guard); - new_records_tx.send(()).unwrap(); - timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap_err(); @@ -790,7 +908,8 @@ mod tests { .unwrap(); drop(state_guard); - new_records_tx.send(()).unwrap(); + let shard_status = (ShardState::Open, Position::offset(0u64)); + shard_status_tx.send(shard_status).unwrap(); timeout(Duration::from_millis(50), fetch_stream.next()) .await @@ -805,21 +924,29 @@ mod tests { .unwrap(); drop(state_guard); - new_records_tx.send(()).unwrap(); + let shard_status = (ShardState::Open, Position::offset(1u64)); + shard_status_tx.send(shard_status).unwrap(); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); - assert_eq!(fetch_response.index_uid, "test-index:0"); - assert_eq!(fetch_response.source_id, "test-source"); - assert_eq!(fetch_response.shard_id, 1); - assert_eq!(fetch_response.from_position_exclusive(), 0u64,); - assert_eq!(fetch_response.to_position_inclusive(), 1u64); + 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_response + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); + assert_eq!( + fetch_payload.to_position_inclusive(), + Position::offset(1u64) + ); + assert_eq!( + fetch_payload .mrecord_batch .as_ref() .unwrap() @@ -827,11 +954,7 @@ mod tests { [14] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, "\0\0test-doc-bar" ); } @@ -860,11 +983,11 @@ mod tests { status: IngesterStatus::Ready, observation_tx, })); - let (_new_records_tx, new_records_rx) = watch::channel(()); + let (_shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); let (mut fetch_stream, fetch_task_handle) = FetchStreamTask::spawn( open_fetch_stream_request, state.clone(), - new_records_rx, + shard_status_rx, 1024, ); let ingest_error = timeout(Duration::from_millis(50), fetch_stream.next()) @@ -901,9 +1024,13 @@ mod tests { status: IngesterStatus::Ready, observation_tx, })); - let (new_records_tx, new_records_rx) = watch::channel(()); - let (mut fetch_stream, _fetch_task_handle) = - FetchStreamTask::spawn(open_fetch_stream_request, state.clone(), new_records_rx, 30); + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let (mut fetch_stream, _fetch_task_handle) = FetchStreamTask::spawn( + open_fetch_stream_request, + state.clone(), + shard_status_rx, + 30, + ); let queue_id = queue_id(&index_uid, &source_id, 1); let mut state_guard = state.write().await; @@ -928,15 +1055,18 @@ mod tests { .unwrap(); drop(state_guard); - new_records_tx.send(()).unwrap(); + let shard_status = (ShardState::Open, Position::offset(2u64)); + shard_status_tx.send(shard_status).unwrap(); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + assert_eq!( - fetch_response + fetch_payload .mrecord_batch .as_ref() .unwrap() @@ -944,21 +1074,19 @@ mod tests { [12, 12] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, "test-doc-footest-doc-bar" ); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + assert_eq!( - fetch_response + fetch_payload .mrecord_batch .as_ref() .unwrap() @@ -966,11 +1094,7 @@ mod tests { [12] ); assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_buffer, + fetch_payload.mrecord_batch.as_ref().unwrap().mrecord_buffer, "test-doc-baz" ); } @@ -1007,12 +1131,12 @@ mod tests { let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); let shard_id: ShardId = 1; - let mut from_position_exclusive = Position::from(0u64); + let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; let ingester_pool = IngesterPool::default(); - let (fetch_response_tx, mut fetch_stream) = ServiceStream::new_bounded(5); + let (fetch_message_tx, mut fetch_stream) = ServiceStream::new_bounded(5); let (service_stream_tx_1, service_stream_1) = ServiceStream::new_unbounded(); let mut ingester_mock_1 = IngesterServiceClient::mock(); @@ -1023,7 +1147,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.from_position_exclusive(), Position::from(0u64)); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) }); @@ -1031,25 +1155,25 @@ mod tests { ingester_pool.insert("test-ingester-1".into(), ingester_1); - let fetch_response = FetchResponseV2 { + let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(1u64)), + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_payload(fetch_payload); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); - let fetch_response = FetchResponseV2 { + let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(1u64)), - to_position_inclusive: Some(Position::Eof), + eof_position: Some(Position::eof(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_eof(fetch_eof); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); fault_tolerant_fetch_stream( client_id, @@ -1059,31 +1183,34 @@ mod tests { &mut from_position_exclusive, &ingester_ids, ingester_pool, - fetch_response_tx, + fetch_message_tx, ) .await; - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + + assert_eq!( + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.to_position_inclusive(), + Position::offset(1u64) ); - assert_eq!(fetch_response.to_position_inclusive(), 1u64); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(1u64) - ); - assert_eq!(fetch_response.to_position_inclusive(), Position::Eof); + let fetch_eof = into_fetch_eof(fetch_message); + + assert_eq!(fetch_eof.eof_position(), Position::eof(1u64)); assert!(timeout(Duration::from_millis(50), fetch_stream.next()) .await @@ -1097,12 +1224,12 @@ mod tests { let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); let shard_id: ShardId = 1; - let mut from_position_exclusive = Position::from(0u64); + let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; let ingester_pool = IngesterPool::default(); - let (fetch_response_tx, mut fetch_stream) = ServiceStream::new_bounded(5); + let (fetch_message_tx, mut fetch_stream) = ServiceStream::new_bounded(5); let (service_stream_tx_1, service_stream_1) = ServiceStream::new_unbounded(); let mut ingester_mock_0 = IngesterServiceClient::mock(); @@ -1113,7 +1240,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.from_position_exclusive(), 0u64); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Err(IngestV2Error::Internal( "open fetch stream error".to_string(), @@ -1129,7 +1256,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.from_position_exclusive(), Position::from(0u64)); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) }); @@ -1138,25 +1265,25 @@ mod tests { ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); - let fetch_response = FetchResponseV2 { + let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(1u64)), + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_payload(fetch_payload); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); - let fetch_response = FetchResponseV2 { + let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(1u64)), - to_position_inclusive: Some(Position::Eof), + eof_position: Some(Position::eof(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_eof(fetch_eof); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); fault_tolerant_fetch_stream( client_id, @@ -1166,31 +1293,34 @@ mod tests { &mut from_position_exclusive, &ingester_ids, ingester_pool, - fetch_response_tx, + fetch_message_tx, ) .await; - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + + assert_eq!( + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.to_position_inclusive(), + Position::offset(1u64) ); - assert_eq!(fetch_response.to_position_inclusive(), 1u64); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(1u64) - ); - assert_eq!(fetch_response.to_position_inclusive(), Position::Eof); + let fetch_eof = into_fetch_eof(fetch_message); + + assert_eq!(fetch_eof.eof_position(), Position::eof(1u64)); assert!(timeout(Duration::from_millis(50), fetch_stream.next()) .await @@ -1204,12 +1334,12 @@ mod tests { let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); let shard_id: ShardId = 1; - let mut from_position_exclusive = Position::from(0u64); + let mut from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; let ingester_pool = IngesterPool::default(); - let (fetch_response_tx, mut fetch_stream) = ServiceStream::new_bounded(5); + let (fetch_message_tx, mut fetch_stream) = ServiceStream::new_bounded(5); let (service_stream_tx_0, service_stream_0) = ServiceStream::new_unbounded(); let (service_stream_tx_1, service_stream_1) = ServiceStream::new_unbounded(); @@ -1221,7 +1351,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.from_position_exclusive(), 0u64); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_0) }); @@ -1235,7 +1365,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.from_position_exclusive(), 1u64); + assert_eq!(request.from_position_exclusive(), Position::offset(1u64)); Ok(service_stream_1) }); @@ -1244,28 +1374,28 @@ mod tests { ingester_pool.insert("test-ingester-0".into(), ingester_0); ingester_pool.insert("test-ingester-1".into(), ingester_1); - let fetch_response = FetchResponseV2 { + let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(1u64)), + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(1u64)), }; - service_stream_tx_0.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_payload(fetch_payload); + service_stream_tx_0.send(Ok(fetch_message)).unwrap(); let ingest_error = IngestV2Error::Internal("fetch stream error".into()); service_stream_tx_0.send(Err(ingest_error)).unwrap(); - let fetch_response = FetchResponseV2 { + let fetch_eof = FetchEof { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(1u64)), - to_position_inclusive: Some(Position::Eof), + eof_position: Some(Position::eof(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_eof(fetch_eof); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); fault_tolerant_fetch_stream( client_id, @@ -1275,31 +1405,34 @@ mod tests { &mut from_position_exclusive, &ingester_ids, ingester_pool, - fetch_response_tx, + fetch_message_tx, ) .await; - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); + assert_eq!( + fetch_payload.to_position_inclusive(), + Position::offset(1u64) ); - assert_eq!(fetch_response.to_position_inclusive(), 1u64); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(1u64) - ); - assert_eq!(fetch_response.to_position_inclusive(), Position::Eof); + let fetch_eof = into_fetch_eof(fetch_message); + + assert_eq!(fetch_eof.eof_position(), Position::eof(1u64)); assert!(timeout(Duration::from_millis(50), fetch_stream.next()) .await @@ -1313,12 +1446,12 @@ mod tests { let index_uid: IndexUid = "test-index:0".into(); let source_id: SourceId = "test-source".into(); let shard_id: ShardId = 1; - let from_position_exclusive = Position::from(0u64); + let from_position_exclusive = Position::offset(0u64); let ingester_ids: Vec = vec!["test-ingester".into()]; let ingester_pool = IngesterPool::default(); - let (fetch_response_tx, mut fetch_stream) = ServiceStream::new_bounded(5); + let (fetch_message_tx, mut fetch_stream) = ServiceStream::new_bounded(5); let (service_stream_tx_1, service_stream_1) = ServiceStream::new_unbounded(); let (service_stream_tx_2, service_stream_2) = ServiceStream::new_unbounded(); @@ -1334,7 +1467,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.from_position_exclusive(), 0u64); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Err(IngestV2Error::Internal( "open fetch stream error".to_string(), @@ -1348,7 +1481,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.from_position_exclusive(), 0u64); + assert_eq!(request.from_position_exclusive(), Position::offset(0u64)); Ok(service_stream_1) }); @@ -1360,7 +1493,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.from_position_exclusive(), 1u64); + assert_eq!(request.from_position_exclusive(), Position::offset(1u64)); Ok(service_stream_2) }); @@ -1368,28 +1501,30 @@ mod tests { ingester_pool.insert("test-ingester".into(), ingester); - let fetch_response = FetchResponseV2 { + let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(1u64)), + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-foo"]), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(1u64)), }; - service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_payload(fetch_payload); + service_stream_tx_1.send(Ok(fetch_message)).unwrap(); let ingest_error = IngestV2Error::Internal("fetch stream error #1".into()); service_stream_tx_1.send(Err(ingest_error)).unwrap(); - let fetch_response = FetchResponseV2 { + let fetch_payload = FetchPayload { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - mrecord_batch: None, - from_position_exclusive: Some(Position::from(1u64)), - to_position_inclusive: Some(Position::from(2u64)), + mrecord_batch: MRecordBatch::for_test(["\0\0test-doc-bar"]), + from_position_exclusive: Some(Position::offset(1u64)), + to_position_inclusive: Some(Position::offset(2u64)), }; - service_stream_tx_2.send(Ok(fetch_response)).unwrap(); + let fetch_message = FetchMessage::new_payload(fetch_payload); + service_stream_tx_2.send(Ok(fetch_message)).unwrap(); let ingest_error = IngestV2Error::Internal("fetch stream error #2".into()); service_stream_tx_2.send(Err(ingest_error)).unwrap(); @@ -1403,7 +1538,7 @@ mod tests { ingester_ids, ingester_pool, retry_params, - fetch_response_tx, + fetch_message_tx, ) .await; @@ -1417,16 +1552,21 @@ mod tests { matches!(ingest_error, IngestV2Error::Internal(message) if message == "open fetch stream error") ); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); + assert_eq!( + fetch_payload.to_position_inclusive(), + Position::offset(1u64) ); - assert_eq!(fetch_response.to_position_inclusive(), Position::from(1u64)); let fetch_stream_error = timeout(Duration::from_millis(50), fetch_stream.next()) .await @@ -1437,16 +1577,21 @@ mod tests { matches!(fetch_stream_error.ingest_error, IngestV2Error::Internal(message) if message == "fetch stream error #1") ); - let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) + let fetch_message = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); + let fetch_payload = into_fetch_payload(fetch_message); + + assert_eq!( + fetch_payload.from_position_exclusive(), + Position::offset(1u64) + ); assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(1u64) + fetch_payload.to_position_inclusive(), + Position::offset(2u64) ); - assert_eq!(fetch_response.to_position_inclusive(), Position::from(2u64)); let fetch_stream_error = timeout(Duration::from_millis(50), fetch_stream.next()) .await diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index fe4d071175a..479928f6dec 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -37,7 +37,7 @@ use quickwit_common::tower::Pool; use quickwit_common::ServiceStream; use quickwit_proto::ingest::ingester::{ AckReplicationMessage, CloseShardsRequest, CloseShardsResponse, DecommissionRequest, - DecommissionResponse, FetchResponseV2, IngesterService, IngesterServiceClient, + DecommissionResponse, FetchMessage, IngesterService, IngesterServiceClient, IngesterServiceStream, IngesterStatus, InitShardsRequest, InitShardsResponse, ObservationMessage, OpenFetchStreamRequest, OpenObservationStreamRequest, OpenReplicationStreamRequest, OpenReplicationStreamResponse, PersistFailure, @@ -53,7 +53,7 @@ use tracing::{debug, error, info, warn}; use super::fetch::FetchStreamTask; use super::models::IngesterShard; use super::mrecord::MRecord; -use super::mrecordlog_utils::{append_eof_record_if_necessary, check_enough_capacity}; +use super::mrecordlog_utils::check_enough_capacity; use super::rate_meter::RateMeter; use super::replication::{ ReplicationClient, ReplicationStreamTask, ReplicationStreamTaskHandle, ReplicationTask, @@ -61,7 +61,7 @@ use super::replication::{ }; use super::IngesterPool; use crate::ingest_v2::broadcast::BroadcastLocalShardsTask; -use crate::ingest_v2::mrecordlog_utils::get_truncation_position; +use crate::ingest_v2::mrecordlog_utils::{delete_queue, queue_position_range}; use crate::metrics::INGEST_METRICS; use crate::{estimate_size, FollowerId, LeaderId}; @@ -171,9 +171,7 @@ impl Ingester { if state.status != IngesterStatus::Decommissioning { return; } - if state.shards.values().all(|shard| { - shard.shard_state.is_closed() && shard.truncation_position_inclusive == Position::Eof - }) { + if state.shards.values().all(|shard| shard.is_indexed()) { info!("ingester fully decommissioned"); state.status = IngesterStatus::Decommissioned; @@ -187,6 +185,9 @@ impl Ingester { } } + /// During the initialization of the ingester, we list all the queues contained in + /// the write-ahead log. Empty queues are deleted, while non-empty queues are recovered. + /// However, the corresponding shards are closed and become read-only. async fn init(&self) -> IngestV2Result<()> { let mut state_guard = self.state.write().await; @@ -196,18 +197,34 @@ impl Ingester { .map(|queue_id| queue_id.to_string()) .collect(); - if queue_ids.is_empty() { - return Ok(()); + if !queue_ids.is_empty() { + info!("recovering {} shard(s)", queue_ids.len()); } - info!("closing {} shard(s)", queue_ids.len()); + let mut num_closed_shards = 0; + let mut num_deleted_shards = 0; for queue_id in queue_ids { - append_eof_record_if_necessary(&mut state_guard.mrecordlog, &queue_id).await; + let Some(position_range) = queue_position_range(&state_guard.mrecordlog, &queue_id) + else { + // Delete empty queue. + delete_queue(&mut state_guard.mrecordlog, &queue_id) + .await + .expect("TODO: handle IO error"); - let truncation_position = get_truncation_position(&state_guard.mrecordlog, &queue_id) - .unwrap_or(Position::Eof); - let solo_shard = - IngesterShard::new_solo(ShardState::Closed, Position::Eof, truncation_position); + num_deleted_shards += 1; + continue; + }; + let replication_position_inclusive = Position::offset(*position_range.end()); + let truncation_position_inclusive = if *position_range.start() == 0 { + Position::Beginning + } else { + Position::offset(*position_range.start() - 1) + }; + let solo_shard = IngesterShard::new_solo( + ShardState::Closed, + replication_position_inclusive, + truncation_position_inclusive, + ); state_guard.shards.insert(queue_id.clone(), solo_shard); let rate_limiter = RateLimiter::from_settings(self.rate_limiter_settings); @@ -215,6 +232,14 @@ impl Ingester { state_guard .rate_trackers .insert(queue_id, (rate_limiter, rate_meter)); + + num_closed_shards += 1; + } + if num_closed_shards > 0 { + info!("recovered and closed {num_closed_shards} shard(s)"); + } + if num_deleted_shards > 0 { + info!("deleted {num_deleted_shards} empty shard(s)"); } Ok(()) } @@ -338,18 +363,6 @@ impl Ingester { entry.insert(replication_stream_task_handle); Ok(replication_client) } - - async fn close_shards_inner(&self, state: &mut IngesterState, queue_ids: &[QueueId]) { - for queue_id in queue_ids { - append_eof_record_if_necessary(&mut state.mrecordlog, queue_id).await; - - if let Some(shard) = state.shards.get_mut(queue_id) { - shard.shard_state = ShardState::Closed; - shard.notify_new_records(); - } - } - // TODO: Handle replicated shards. - } } #[async_trait] @@ -503,7 +516,9 @@ impl IngesterService for Ingester { .await .expect("TODO") // TODO: Io error, close shard? } - .into(); + .map(Position::offset) + .expect("records should not be empty"); + INGEST_METRICS.ingested_num_bytes.inc_by(batch_num_bytes); INGEST_METRICS.ingested_num_docs.inc_by(batch_num_docs); @@ -672,21 +687,21 @@ impl IngesterService for Ingester { async fn open_fetch_stream( &mut self, open_fetch_stream_request: OpenFetchStreamRequest, - ) -> IngestV2Result>> { + ) -> IngestV2Result>> { let queue_id = open_fetch_stream_request.queue_id(); - let new_records_rx = self + let shard_status_rx = self .state .read() .await .shards .get(&queue_id) .ok_or_else(|| IngestV2Error::Internal("shard not found".to_string()))? - .new_records_rx + .shard_status_rx .clone(); let (service_stream, _fetch_task_handle) = FetchStreamTask::spawn( open_fetch_stream_request, self.state.clone(), - new_records_rx, + shard_status_rx, FetchStreamTask::DEFAULT_BATCH_NUM_BYTES, ); Ok(service_stream) @@ -747,17 +762,14 @@ impl IngesterService for Ingester { for subrequest in truncate_shards_request.subrequests { let queue_id = subrequest.queue_id(); + let to_position_inclusive = subrequest.to_position_inclusive(); - let truncate_position_opt = match subrequest.to_position_inclusive() { - Position::Beginning => None, - Position::Offset(offset) => offset.as_u64(), - Position::Eof => state_guard - .mrecordlog - .last_position(&queue_id) - .ok() - .flatten(), - }; - if let Some(truncate_position) = truncate_position_opt { + if to_position_inclusive.is_eof() { + delete_queue(&mut state_guard.mrecordlog, &queue_id) + .await + .expect("TODO: handle IO error"); + state_guard.shards.remove(&queue_id); + } else if let Some(truncate_position) = to_position_inclusive.as_u64() { match state_guard .mrecordlog .truncate(&queue_id, truncate_position) @@ -768,13 +780,10 @@ impl IngesterService for Ingester { error!("failed to truncate queue `{queue_id}`: {error}"); continue; } - } - // If we successfully truncated the queue at EOF, it means that the shard is closed - // and fully indexed. - if subrequest.to_position_inclusive() == Position::Eof { - if let Some(shard) = state_guard.shards.get_mut(&queue_id) { - shard.truncation_position_inclusive = Position::Eof - } + }; + if let Some(shard) = state_guard.shards.get_mut(&queue_id) { + shard.truncation_position_inclusive = to_position_inclusive; + shard.notify_shard_status(); } } } @@ -787,16 +796,16 @@ impl IngesterService for Ingester { &mut self, close_shards_request: CloseShardsRequest, ) -> IngestV2Result { - let queue_ids: Vec = close_shards_request - .shards - .iter() - .flat_map(|shards| shards.queue_ids()) - .collect(); - let mut state_guard = self.state.write().await; - self.close_shards_inner(&mut state_guard, &queue_ids).await; - + for shard_ids in close_shards_request.shards { + for queue_id in shard_ids.queue_ids() { + if let Some(shard) = state_guard.shards.get_mut(&queue_id) { + shard.shard_state = ShardState::Closed; + shard.notify_shard_status(); + } + } + } Ok(CloseShardsResponse {}) } @@ -807,9 +816,10 @@ impl IngesterService for Ingester { info!("decommissioning ingester"); let mut state_guard = self.state.write().await; - let queue_ids: Vec = state_guard.shards.keys().cloned().collect(); - self.close_shards_inner(&mut state_guard, &queue_ids).await; - + for shard in state_guard.shards.values_mut() { + shard.shard_state = ShardState::Closed; + shard.notify_shard_status(); + } state_guard.status = IngesterStatus::Decommissioning; self.check_decommissioning_status(&mut state_guard); @@ -878,7 +888,7 @@ mod tests { use super::*; use crate::ingest_v2::broadcast::ShardInfos; - use crate::ingest_v2::mrecord::is_eof_mrecord; + use crate::ingest_v2::fetch::tests::{into_fetch_eof, into_fetch_payload}; use crate::ingest_v2::test_utils::MultiRecordLogTestExt; pub(super) struct IngesterForTest { @@ -1047,49 +1057,20 @@ mod tests { ingester.init().await.unwrap(); - // It should only append EOF records if necessary. - ingester.init().await.unwrap(); - let state_guard = ingester.state.read().await; - assert_eq!(state_guard.shards.len(), 3); - - let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); - solo_shard_01.assert_is_solo(); - solo_shard_01.assert_is_closed(); - solo_shard_01.assert_replication_position(Position::Eof); - solo_shard_01.assert_truncation_position(Position::Eof); - - assert!(state_guard.rate_trackers.contains_key(&queue_id_01)); - - state_guard - .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(1, "\0\x02")]); + assert_eq!(state_guard.shards.len(), 1); let solo_shard_02 = state_guard.shards.get(&queue_id_02).unwrap(); solo_shard_02.assert_is_solo(); solo_shard_02.assert_is_closed(); - solo_shard_02.assert_replication_position(Position::Eof); - solo_shard_02.assert_truncation_position(0u64); - - assert!(state_guard.rate_trackers.contains_key(&queue_id_02)); - - state_guard.mrecordlog.assert_records_eq( - &queue_id_02, - .., - &[(1, "\0\0test-doc-bar"), (2, "\0\x02")], - ); - - let solo_shard_03 = state_guard.shards.get(&queue_id_03).unwrap(); - solo_shard_03.assert_is_solo(); - solo_shard_03.assert_is_closed(); - solo_shard_03.assert_replication_position(Position::Eof); - solo_shard_03.assert_truncation_position(Position::Eof); - - assert!(state_guard.rate_trackers.contains_key(&queue_id_02)); + solo_shard_02.assert_replication_position(Position::offset(1u64)); + solo_shard_02.assert_truncation_position(Position::offset(0u64)); state_guard .mrecordlog - .assert_records_eq(&queue_id_03, .., &[(0, "\0\x02")]); + .assert_records_eq(&queue_id_02, .., &[(1, "\0\0test-doc-bar")]); + + state_guard.rate_trackers.contains_key(&queue_id_02); } #[tokio::test] @@ -1099,7 +1080,8 @@ mod tests { let mut state_guard = ingester.state.write().await; let queue_id_01 = queue_id("test-index:0", "test-source", 1); - let shard = IngesterShard::new_solo(ShardState::Open, Position::Beginning, Position::Eof); + let shard = + IngesterShard::new_solo(ShardState::Open, Position::Beginning, Position::Beginning); state_guard.shards.insert(queue_id_01.clone(), shard); let rate_limiter = RateLimiter::from_settings(RateLimiterSettings::default()); @@ -1214,7 +1196,7 @@ mod tests { assert_eq!(persist_success_0.shard_id, 1); assert_eq!( persist_success_0.replication_position_inclusive, - Some(Position::from(1u64)) + Some(Position::offset(1u64)) ); let persist_success_1 = &persist_response.successes[1]; @@ -1224,7 +1206,7 @@ mod tests { assert_eq!(persist_success_1.shard_id, 1); assert_eq!( persist_success_1.replication_position_inclusive, - Some(Position::from(2u64)) + Some(Position::offset(2u64)) ); let state_guard = ingester.state.read().await; @@ -1234,7 +1216,7 @@ mod tests { let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); solo_shard_01.assert_is_solo(); solo_shard_01.assert_is_open(); - solo_shard_01.assert_replication_position(1u64); + solo_shard_01.assert_replication_position(Position::offset(1u64)); state_guard.mrecordlog.assert_records_eq( &queue_id_01, @@ -1246,7 +1228,7 @@ mod tests { let solo_shard_11 = state_guard.shards.get(&queue_id_11).unwrap(); solo_shard_11.assert_is_solo(); solo_shard_11.assert_is_open(); - solo_shard_11.assert_replication_position(2u64); + solo_shard_11.assert_replication_position(Position::offset(2u64)); state_guard.mrecordlog.assert_records_eq( &queue_id_11, @@ -1371,7 +1353,7 @@ mod tests { assert_eq!(persist_success_0.shard_id, 1); assert_eq!( persist_success_0.replication_position_inclusive, - Some(Position::from(1u64)) + Some(Position::offset(1u64)) ); let persist_success_1 = &persist_response.successes[1]; @@ -1381,7 +1363,7 @@ mod tests { assert_eq!(persist_success_1.shard_id, 1); assert_eq!( persist_success_1.replication_position_inclusive, - Some(Position::from(2u64)) + Some(Position::offset(2u64)) ); let leader_state_guard = leader.state.read().await; @@ -1391,7 +1373,7 @@ mod tests { let primary_shard_01 = leader_state_guard.shards.get(&queue_id_01).unwrap(); primary_shard_01.assert_is_primary(); primary_shard_01.assert_is_open(); - primary_shard_01.assert_replication_position(1u64); + primary_shard_01.assert_replication_position(Position::offset(1u64)); leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, @@ -1403,7 +1385,7 @@ mod tests { let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); primary_shard_11.assert_is_primary(); primary_shard_11.assert_is_open(); - primary_shard_11.assert_replication_position(2u64); + primary_shard_11.assert_replication_position(Position::offset(2u64)); leader_state_guard.mrecordlog.assert_records_eq( &queue_id_11, @@ -1421,7 +1403,7 @@ mod tests { let replica_shard_01 = follower_state_guard.shards.get(&queue_id_01).unwrap(); replica_shard_01.assert_is_replica(); replica_shard_01.assert_is_open(); - replica_shard_01.assert_replication_position(1u64); + replica_shard_01.assert_replication_position(Position::offset(1u64)); follower_state_guard.mrecordlog.assert_records_eq( &queue_id_01, @@ -1432,7 +1414,7 @@ mod tests { let replica_shard_11 = follower_state_guard.shards.get(&queue_id_11).unwrap(); replica_shard_11.assert_is_replica(); replica_shard_11.assert_is_open(); - replica_shard_11.assert_replication_position(2u64); + replica_shard_11.assert_replication_position(Position::offset(2u64)); follower_state_guard.mrecordlog.assert_records_eq( &queue_id_11, @@ -1555,7 +1537,7 @@ mod tests { assert_eq!(persist_success_0.shard_id, 1); assert_eq!( persist_success_0.replication_position_inclusive, - Some(Position::from(0u64)) + Some(Position::offset(0u64)) ); let persist_success_1 = &persist_response.successes[1]; @@ -1565,7 +1547,7 @@ mod tests { assert_eq!(persist_success_1.shard_id, 1); assert_eq!( persist_success_1.replication_position_inclusive, - Some(Position::from(1u64)) + Some(Position::offset(1u64)) ); let leader_state_guard = leader.state.read().await; @@ -1575,7 +1557,7 @@ mod tests { let primary_shard_01 = leader_state_guard.shards.get(&queue_id_01).unwrap(); primary_shard_01.assert_is_primary(); primary_shard_01.assert_is_open(); - primary_shard_01.assert_replication_position(0u64); + primary_shard_01.assert_replication_position(Position::offset(0u64)); leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, @@ -1587,7 +1569,7 @@ mod tests { let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); primary_shard_11.assert_is_primary(); primary_shard_11.assert_is_open(); - primary_shard_11.assert_replication_position(1u64); + primary_shard_11.assert_replication_position(Position::offset(1u64)); leader_state_guard.mrecordlog.assert_records_eq( &queue_id_11, @@ -1601,7 +1583,7 @@ mod tests { let replica_shard_01 = follower_state_guard.shards.get(&queue_id_01).unwrap(); replica_shard_01.assert_is_replica(); replica_shard_01.assert_is_open(); - replica_shard_01.assert_replication_position(0u64); + replica_shard_01.assert_replication_position(Position::offset(0u64)); follower_state_guard.mrecordlog.assert_records_eq( &queue_id_01, @@ -1612,7 +1594,7 @@ mod tests { let replica_shard_11 = follower_state_guard.shards.get(&queue_id_11).unwrap(); replica_shard_11.assert_is_replica(); replica_shard_11.assert_is_open(); - replica_shard_11.assert_replication_position(1u64); + replica_shard_11.assert_replication_position(Position::offset(1u64)); follower_state_guard.mrecordlog.assert_records_eq( &queue_id_11, @@ -1801,58 +1783,36 @@ mod tests { #[tokio::test] async fn test_ingester_open_fetch_stream() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; - let init_shards_request = InitShardsRequest { - shards: vec![ - Shard { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - shard_state: ShardState::Open as i32, - leader_id: ingester_ctx.node_id.to_string(), - ..Default::default() - }, - Shard { - index_uid: "test-index:1".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - shard_state: ShardState::Open as i32, - leader_id: ingester_ctx.node_id.to_string(), - ..Default::default() - }, - ], + let shard = Shard { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + shard_state: ShardState::Open as i32, + ..Default::default() }; - ingester.init_shards(init_shards_request).await.unwrap(); + let queue_id = queue_id("test-index:0", "test-source", 1); - let persist_request = PersistRequest { - leader_id: ingester_ctx.node_id.to_string(), - commit_type: CommitTypeV2::Auto as i32, - subrequests: vec![ - PersistSubrequest { - subrequest_id: 0, - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), - }, - PersistSubrequest { - subrequest_id: 1, - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 2, - follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-020"])), - }, - ], - }; - ingester.persist(persist_request).await.unwrap(); + let mut state_guard = ingester.state.write().await; + + ingester + .init_primary_shard(&mut state_guard, shard) + .await + .unwrap(); + + let records = [MRecord::new_doc("test-doc-foo").encode()].into_iter(); - let client_id = "test-client".to_string(); + state_guard + .mrecordlog + .append_records(&queue_id, None, records) + .await + .unwrap(); + + drop(state_guard); let open_fetch_stream_request = OpenFetchStreamRequest { - client_id: client_id.clone(), + client_id: "test-client".to_string(), index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -1864,46 +1824,57 @@ mod tests { .unwrap(); let fetch_response = fetch_stream.next().await.unwrap().unwrap(); + let fetch_payload = into_fetch_payload(fetch_response); + + assert_eq!(fetch_payload.from_position_exclusive(), Position::Beginning); assert_eq!( - fetch_response.from_position_exclusive(), - Position::Beginning + fetch_payload.to_position_inclusive(), + Position::offset(0u64) ); - assert_eq!(fetch_response.to_position_inclusive(), Position::from(0u64)); - let mrecord_batch = fetch_response.mrecord_batch.unwrap(); + let mrecord_batch = fetch_payload.mrecord_batch.unwrap(); assert_eq!( mrecord_batch.mrecord_buffer, - Bytes::from_static(b"\0\0test-doc-010") + Bytes::from_static(b"\0\0test-doc-foo") ); assert_eq!(mrecord_batch.mrecord_lengths, [14]); - let persist_request = PersistRequest { - leader_id: ingester_ctx.node_id.to_string(), - commit_type: CommitTypeV2::Auto as i32, - subrequests: vec![PersistSubrequest { - subrequest_id: 0, - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-011", "test-doc-012"])), - }], - }; - ingester.persist(persist_request).await.unwrap(); + let mut state_guard = ingester.state.write().await; + + let records = [MRecord::new_doc("test-doc-bar").encode()].into_iter(); + + state_guard + .mrecordlog + .append_records(&queue_id, None, records) + .await + .unwrap(); + + state_guard + .shards + .get(&queue_id) + .unwrap() + .notify_shard_status(); + + drop(state_guard); let fetch_response = fetch_stream.next().await.unwrap().unwrap(); + let fetch_payload = into_fetch_payload(fetch_response); + assert_eq!( - fetch_response.from_position_exclusive(), - Position::from(0u64) + fetch_payload.from_position_exclusive(), + Position::offset(0u64) + ); + assert_eq!( + fetch_payload.to_position_inclusive(), + Position::offset(1u64) ); - assert_eq!(fetch_response.to_position_inclusive(), Position::from(2u64)); - let mrecord_batch = fetch_response.mrecord_batch.unwrap(); + let mrecord_batch = fetch_payload.mrecord_batch.unwrap(); assert_eq!( mrecord_batch.mrecord_buffer, - Bytes::from_static(b"\0\0test-doc-011\0\0test-doc-012") + Bytes::from_static(b"\0\0test-doc-bar") ); - assert_eq!(mrecord_batch.mrecord_lengths, [14, 14]); + assert_eq!(mrecord_batch.mrecord_lengths, [14]); } #[tokio::test] @@ -1923,7 +1894,7 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, - shard_state: ShardState::Open as i32, + shard_state: ShardState::Closed as i32, ..Default::default() }; let queue_id_02 = queue_id("test-index:0", "test-source", 2); @@ -1940,8 +1911,8 @@ mod tests { .unwrap(); let records = [ - MRecord::new_doc("test-doc-010").encode(), - MRecord::new_doc("test-doc-011").encode(), + MRecord::new_doc("test-doc-foo").encode(), + MRecord::new_doc("test-doc-bar").encode(), ] .into_iter(); @@ -1951,7 +1922,7 @@ mod tests { .await .unwrap(); - let records = [MRecord::new_doc("test-doc-020").encode()].into_iter(); + let records = [MRecord::new_doc("test-doc-baz").encode()].into_iter(); state_guard .mrecordlog @@ -1968,19 +1939,19 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - to_position_inclusive: Some(Position::from(0u64)), + to_position_inclusive: Some(Position::offset(0u64)), }, TruncateShardsSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, - to_position_inclusive: Some(Position::Eof), + to_position_inclusive: Some(Position::eof(0u64)), }, TruncateShardsSubrequest { index_uid: "test-index:1337".to_string(), source_id: "test-source".to_string(), - shard_id: 1, - to_position_inclusive: Some(Position::from(1337u64)), + shard_id: 1337, + to_position_inclusive: Some(Position::offset(1337u64)), }, ], }; @@ -1990,78 +1961,39 @@ mod tests { .unwrap(); let state_guard = ingester.state.read().await; - assert_eq!(state_guard.shards.len(), 2); + assert_eq!(state_guard.shards.len(), 1); assert!(state_guard.shards.contains_key(&queue_id_01)); state_guard .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(1, "\0\0test-doc-011")]); + .assert_records_eq(&queue_id_01, .., &[(1, "\0\0test-doc-bar")]); - assert!(state_guard.shards.contains_key(&queue_id_02)); - state_guard - .mrecordlog - .assert_records_eq(&queue_id_02, .., &[]); + assert!(!state_guard.shards.contains_key(&queue_id_02)); + assert!(!state_guard.mrecordlog.queue_exists(&queue_id_02)); } #[tokio::test] async fn test_ingester_close_shards() { let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; - let shard_01 = Shard { + let shard = Shard { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, shard_state: ShardState::Open as i32, ..Default::default() }; - let queue_id_01 = queue_id("test-index:0", "test-source", 1); - - let shard_02 = Shard { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 2, - shard_state: ShardState::Open as i32, - ..Default::default() - }; - let queue_id_02 = queue_id("test-index:0", "test-source", 2); - - let shard_13 = Shard { - index_uid: "test-index:1".to_string(), - source_id: "test-source".to_string(), - shard_id: 3, - shard_state: ShardState::Open as i32, - ..Default::default() - }; - let queue_id_13 = queue_id("test-index:1", "test-source", 3); + let queue_id = queue_id("test-index:0", "test-source", 1); let mut state_guard = ingester.state.write().await; - - for (shard, queue_id) in [ - (shard_01, &queue_id_01), - (shard_02, &queue_id_02), - (shard_13, &queue_id_13), - ] { - ingester - .init_primary_shard(&mut state_guard, shard) - .await - .unwrap(); - let records = [ - MRecord::new_doc("test-doc-010").encode(), - MRecord::new_doc("test-doc-011").encode(), - ] - .into_iter(); - state_guard - .mrecordlog - .append_records(queue_id, None, records) - .await - .unwrap(); - } - + ingester + .init_primary_shard(&mut state_guard, shard) + .await + .unwrap(); drop(state_guard); - let client_id = "test-client".to_string(); let open_fetch_stream_request = OpenFetchStreamRequest { - client_id: client_id.clone(), + client_id: "test-client".to_string(), index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, @@ -2071,63 +2003,29 @@ mod tests { .open_fetch_stream(open_fetch_stream_request) .await .unwrap(); - let fetch_response = fetch_stream.next().await.unwrap().unwrap(); - assert_eq!( - fetch_response.from_position_exclusive(), - Position::Beginning - ); - let close_shard_1 = ShardIds { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_ids: vec![1, 2], - }; - let close_shard_2 = ShardIds { - index_uid: "test-index:1".to_string(), - source_id: "test-source".to_string(), - shard_ids: vec![3], - }; - let close_shard_with_no_queue = ShardIds { - index_uid: "test-index:2".to_string(), - source_id: "test-source".to_string(), - shard_ids: vec![4], - }; - let closed_shards = vec![ - close_shard_1.clone(), - close_shard_2.clone(), - close_shard_with_no_queue, - ]; let close_shards_request = CloseShardsRequest { - shards: closed_shards.clone(), + shards: vec![ShardIds { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_ids: vec![1, 1337], + }], }; ingester.close_shards(close_shards_request).await.unwrap(); - // Check that shards are closed and EOF records are appended. let state_guard = ingester.state.read().await; - for shard in state_guard.shards.values() { - shard.assert_is_closed(); - } - for closed_shards in [&close_shard_1, &close_shard_2] { - for queue_id in closed_shards.queue_ids() { - let last_position = state_guard - .mrecordlog - .range(&queue_id, ..) - .unwrap() - .last() - .unwrap(); - assert!(is_eof_mrecord(&last_position.1)); - } - } + let shard = state_guard.shards.get(&queue_id).unwrap(); + shard.assert_is_closed(); - // Check that fetch task is notified. - // Note: fetch stream should not block if the close shard call notified the fetch task. let fetch_response = tokio::time::timeout(std::time::Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!(fetch_response.to_position_inclusive(), Position::Eof); + let fetch_eof = into_fetch_eof(fetch_response); + + assert_eq!(fetch_eof.eof_position(), Position::Beginning.as_eof()); } #[tokio::test] @@ -2185,13 +2083,17 @@ mod tests { state_guard.shards.insert( queue_id_01.clone(), - IngesterShard::new_solo(ShardState::Closed, Position::Eof, Position::Beginning), + IngesterShard::new_solo( + ShardState::Closed, + Position::offset(12u64), + Position::Beginning, + ), ); ingester.check_decommissioning_status(&mut state_guard); assert_eq!(state_guard.status, IngesterStatus::Decommissioning); let shard = state_guard.shards.get_mut(&queue_id_01).unwrap(); - shard.truncation_position_inclusive = Position::Eof; + shard.truncation_position_inclusive = Position::Beginning.as_eof(); ingester.check_decommissioning_status(&mut state_guard); assert_eq!(state_guard.status, IngesterStatus::Decommissioned); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/models.rs b/quickwit/quickwit-ingest/src/ingest_v2/models.rs index 9656b387f66..c1ec3d13d22 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/models.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/models.rs @@ -31,6 +31,9 @@ pub(super) enum IngesterShardType { Solo, } +/// Status of a shard: state + position of the last record written. +pub(super) type ShardStatus = (ShardState, Position); + #[derive(Debug)] pub(super) struct IngesterShard { pub shard_type: IngesterShardType, @@ -39,8 +42,8 @@ pub(super) struct IngesterShard { pub replication_position_inclusive: Position, /// Position up to which the shard has been truncated. pub truncation_position_inclusive: Position, - pub new_records_tx: watch::Sender<()>, - pub new_records_rx: watch::Receiver<()>, + pub shard_status_tx: watch::Sender, + pub shard_status_rx: watch::Receiver, } impl IngesterShard { @@ -50,14 +53,15 @@ impl IngesterShard { replication_position_inclusive: Position, truncation_position_inclusive: Position, ) -> Self { - let (new_records_tx, new_records_rx) = watch::channel(()); + let shard_status = (shard_state, replication_position_inclusive.clone()); + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); Self { shard_type: IngesterShardType::Primary { follower_id }, shard_state, replication_position_inclusive, truncation_position_inclusive, - new_records_tx, - new_records_rx, + shard_status_tx, + shard_status_rx, } } @@ -67,14 +71,15 @@ impl IngesterShard { replication_position_inclusive: Position, truncation_position_inclusive: Position, ) -> Self { - let (new_records_tx, new_records_rx) = watch::channel(()); + let shard_status = (shard_state, replication_position_inclusive.clone()); + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); Self { shard_type: IngesterShardType::Replica { leader_id }, shard_state, replication_position_inclusive, truncation_position_inclusive, - new_records_tx, - new_records_rx, + shard_status_tx, + shard_status_rx, } } @@ -83,17 +88,22 @@ impl IngesterShard { replication_position_inclusive: Position, truncation_position_inclusive: Position, ) -> Self { - let (new_records_tx, new_records_rx) = watch::channel(()); + let shard_status = (shard_state, replication_position_inclusive.clone()); + let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); Self { shard_type: IngesterShardType::Solo, shard_state, replication_position_inclusive, truncation_position_inclusive, - new_records_tx, - new_records_rx, + shard_status_tx, + shard_status_rx, } } + pub fn is_indexed(&self) -> bool { + self.shard_state.is_closed() && self.truncation_position_inclusive.is_eof() + } + pub fn is_replica(&self) -> bool { matches!(self.shard_type, IngesterShardType::Replica { .. }) } @@ -106,10 +116,14 @@ impl IngesterShard { } } - pub fn notify_new_records(&mut self) { - // `new_records_tx` is guaranteed to be open because `self` also holds a receiver. - self.new_records_tx - .send(()) + pub fn notify_shard_status(&self) { + // `shard_status_tx` is guaranteed to be open because `self` also holds a receiver. + let shard_status = ( + self.shard_state, + self.replication_position_inclusive.clone(), + ); + self.shard_status_tx + .send(shard_status) .expect("channel should be open"); } @@ -118,7 +132,7 @@ impl IngesterShard { return; } self.replication_position_inclusive = replication_position_inclusive; - self.notify_new_records(); + self.notify_shard_status(); } } @@ -153,12 +167,7 @@ mod tests { } #[track_caller] - pub fn assert_replication_position( - &self, - expected_replication_position: impl Into, - ) { - let expected_replication_position = expected_replication_position.into(); - + pub fn assert_replication_position(&self, expected_replication_position: Position) { assert_eq!( self.replication_position_inclusive, expected_replication_position, "expected replication position at `{:?}`, got `{:?}`", @@ -167,12 +176,7 @@ mod tests { } #[track_caller] - pub fn assert_truncation_position( - &self, - expected_truncation_position: impl Into, - ) { - let expected_truncation_position = expected_truncation_position.into(); - + pub fn assert_truncation_position(&self, expected_truncation_position: Position) { assert_eq!( self.truncation_position_inclusive, expected_truncation_position, "expected truncation position at `{:?}`, got `{:?}`", @@ -186,43 +190,66 @@ mod tests { let primary_shard = IngesterShard::new_primary( "test-follower".into(), ShardState::Closed, - Position::from(42u64), - Position::Eof, + Position::offset(42u64), + Position::Beginning, ); assert!(matches!( - primary_shard.shard_type, - IngesterShardType::Primary { .. } + &primary_shard.shard_type, + IngesterShardType::Primary { follower_id } if *follower_id == "test-follower" )); + assert!(!primary_shard.is_replica()); assert_eq!(primary_shard.shard_state, ShardState::Closed); - assert_eq!(primary_shard.truncation_position_inclusive, Position::Eof); assert_eq!( primary_shard.replication_position_inclusive, - Position::from(42u64) + Position::offset(42u64) + ); + assert_eq!( + primary_shard.truncation_position_inclusive, + Position::Beginning ); } #[test] fn test_new_replica_shard() { - let solo_shard = IngesterShard::new_solo(ShardState::Closed, 42u64.into(), Position::Eof); - assert_eq!(solo_shard.shard_type, IngesterShardType::Solo); - assert_eq!(solo_shard.shard_state, ShardState::Closed); - assert_eq!(solo_shard.truncation_position_inclusive, Position::Eof); + let replica_shard = IngesterShard::new_replica( + "test-leader".into(), + ShardState::Closed, + Position::offset(42u64), + Position::Beginning, + ); + assert!(matches!( + &replica_shard.shard_type, + IngesterShardType::Replica { leader_id } if *leader_id == "test-leader" + )); + assert!(replica_shard.is_replica()); + assert_eq!(replica_shard.shard_state, ShardState::Closed); assert_eq!( - solo_shard.replication_position_inclusive, - Position::from(42u64) + replica_shard.replication_position_inclusive, + Position::offset(42u64) + ); + assert_eq!( + replica_shard.truncation_position_inclusive, + Position::Beginning ); } #[test] fn test_new_solo_shard() { - let solo_shard = - IngesterShard::new_solo(ShardState::Closed, Position::from(42u64), Position::Eof); + let solo_shard = IngesterShard::new_solo( + ShardState::Closed, + Position::offset(42u64), + Position::Beginning, + ); assert_eq!(solo_shard.shard_type, IngesterShardType::Solo); + assert!(!solo_shard.is_replica()); assert_eq!(solo_shard.shard_state, ShardState::Closed); - assert_eq!(solo_shard.truncation_position_inclusive, Position::Eof); assert_eq!( solo_shard.replication_position_inclusive, - Position::from(42u64) + Position::offset(42u64) + ); + assert_eq!( + solo_shard.truncation_position_inclusive, + Position::Beginning ); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs b/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs index d73b7b83dc8..2a5d4acf6a3 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs @@ -19,6 +19,7 @@ use bytes::{Buf, Bytes}; use quickwit_proto::ingest::MRecordBatch; +use tracing::warn; /// The first byte of a [`MRecord`] is the version of the record header. #[derive(Debug)] @@ -37,15 +38,10 @@ const DOC_HEADER_V0: &[u8; MRECORD_HEADER_LEN] = &[HeaderVersion::V0 as u8, 0]; /// `Commit` header v0 composed of the header version and the `Commit = 1` record type. const COMMIT_HEADER_V0: &[u8; MRECORD_HEADER_LEN] = &[HeaderVersion::V0 as u8, 1]; -/// `Eof` header v0 composed of the header version and the `Eof = 2` record type. -const EOF_HEADER_V0: &[u8; MRECORD_HEADER_LEN] = &[HeaderVersion::V0 as u8, 2]; - #[derive(Debug, Clone, Eq, PartialEq)] pub enum MRecord { Doc(Bytes), Commit, - Eof, - Unknown, } impl MRecord { @@ -53,26 +49,28 @@ impl MRecord { match &self { Self::Doc(doc) => DOC_HEADER_V0.chain(doc.clone()), Self::Commit => COMMIT_HEADER_V0.chain(Bytes::new()), - Self::Eof => EOF_HEADER_V0.chain(Bytes::new()), - Self::Unknown => panic!("unknown mrecord type should not be encoded"), } } - pub fn decode(mut buf: impl Buf) -> Self { + pub fn decode(mut buf: impl Buf) -> Option { let header_version = buf.get_u8(); if header_version != HeaderVersion::V0 as u8 { - return Self::Unknown; + warn!("unknown mrecord header version `{header_version}`"); + return None; } - match buf.get_u8() { + let mrecord = match buf.get_u8() { 0 => { let doc = buf.copy_to_bytes(buf.remaining()); Self::Doc(doc) } 1 => Self::Commit, - 2 => Self::Eof, - _ => Self::Unknown, - } + other => { + warn!("unknown mrecord type `{other}`"); + return None; + } + }; + Some(mrecord) } #[cfg(any(test, feature = "testsuite"))] @@ -82,11 +80,7 @@ impl MRecord { } pub fn decoded_mrecords(mrecord_batch: &MRecordBatch) -> impl Iterator + '_ { - mrecord_batch.encoded_mrecords().map(MRecord::decode) -} - -pub(super) fn is_eof_mrecord(mrecord: &[u8]) -> bool { - mrecord == EOF_HEADER_V0 + mrecord_batch.encoded_mrecords().flat_map(MRecord::decode) } #[cfg(test)] @@ -97,7 +91,7 @@ mod tests { fn test_mrecord_doc_roundtrip() { let record = MRecord::new_doc("hello"); let encoded_record = record.encode(); - let decoded_record = MRecord::decode(encoded_record); + let decoded_record = MRecord::decode(encoded_record).unwrap(); assert_eq!(record, decoded_record); } @@ -105,21 +99,7 @@ mod tests { fn test_mrecord_commit_roundtrip() { let record = MRecord::Commit; let encoded_record = record.encode(); - let decoded_record = MRecord::decode(encoded_record); + let decoded_record = MRecord::decode(encoded_record).unwrap(); assert_eq!(record, decoded_record); } - - #[test] - fn test_mrecord_eof_roundtrip() { - let record = MRecord::Eof; - let encoded_record = record.encode(); - let decoded_record = MRecord::decode(encoded_record); - assert_eq!(record, decoded_record); - } - - #[test] - fn test_mrecord_is_eof_mrecord() { - assert!(is_eof_mrecord(EOF_HEADER_V0)); - assert!(!is_eof_mrecord(COMMIT_HEADER_V0)); - } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs index a7d491080ff..f135680c4f6 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mrecordlog_utils.rs @@ -17,41 +17,13 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +use std::io; +use std::ops::RangeInclusive; + use bytesize::ByteSize; -use mrecordlog::error::{AppendError, MissingQueue}; +use mrecordlog::error::DeleteQueueError; use mrecordlog::MultiRecordLog; -use quickwit_proto::types::{Position, QueueId}; -use tracing::warn; - -use super::mrecord::is_eof_mrecord; -use crate::MRecord; - -/// Appends an EOF record to the queue if it is empty or the last record is not an EOF -/// record. -pub(super) async fn append_eof_record_if_necessary( - mrecordlog: &mut MultiRecordLog, - queue_id: &QueueId, -) { - let should_append_eof_record = match mrecordlog.last_record(queue_id) { - Ok(Some((_, last_mrecord))) => !is_eof_mrecord(&last_mrecord), - Ok(None) => true, - Err(MissingQueue(_)) => { - warn!("failed to append EOF record to queue `{queue_id}`: queue does not exist"); - return; - } - }; - if should_append_eof_record { - match mrecordlog - .append_record(queue_id, None, MRecord::Eof.encode()) - .await - { - Ok(_) | Err(AppendError::MissingQueue(_)) => {} - Err(error) => { - warn!("failed to append EOF record to queue `{queue_id}`: {error}"); - } - } - } -} +use quickwit_proto::types::QueueId; /// Error returned when the mrecordlog does not have enough capacity to store some records. #[derive(Debug, Clone, Copy, thiserror::Error)] @@ -104,62 +76,38 @@ pub(super) fn check_enough_capacity( Ok(()) } -/// Returns the position up to which the queue has been truncated (inclusive) taking into account -/// `Eof` records. Returns `None` if the queue does not exist. -pub(super) fn get_truncation_position( +/// Deletes a queue from the WAL. Returns without error if the queue does not exist. +pub async fn delete_queue(mrecordlog: &mut MultiRecordLog, queue_id: &QueueId) -> io::Result<()> { + match mrecordlog.delete_queue(queue_id).await { + Ok(_) | Err(DeleteQueueError::MissingQueue(_)) => Ok(()), + Err(DeleteQueueError::IoError(error)) => Err(error), + } +} + +/// Returns the first and last position of the records currently stored in the queue. Returns `None` +/// if the queue does not exist or is empty. +pub(super) fn queue_position_range( mrecordlog: &MultiRecordLog, queue_id: &QueueId, -) -> Option { - let Ok(mut mrecords) = mrecordlog.range(queue_id, ..) else { - return None; - }; - let position_opt = if let Some((position, mrecord)) = mrecords.next() { - if is_eof_mrecord(&mrecord) { - return Some(Position::Eof); - } - position.checked_sub(1) - } else { - // The queue exists and is empty. The last position should give us the truncation position. - mrecordlog - .last_position(queue_id) - .expect("queue should exist") - }; - Some(position_opt.map(Position::from).unwrap_or_default()) +) -> Option> { + let first_position = mrecordlog + .range(queue_id, ..) + .ok()? + .next() + .map(|(position, _)| position)?; + + let last_position = mrecordlog + .last_record(queue_id) + .ok()? + .map(|(position, _)| position)?; + + Some(first_position..=last_position) } #[cfg(test)] mod tests { - use bytes::Bytes; - use super::*; - #[tokio::test] - async fn test_append_eof_record_if_necessary() { - let tempdir = tempfile::tempdir().unwrap(); - let mut mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); - - append_eof_record_if_necessary(&mut mrecordlog, &"queue-not-found".to_string()).await; - - mrecordlog.create_queue("test-queue").await.unwrap(); - append_eof_record_if_necessary(&mut mrecordlog, &"test-queue".to_string()).await; - - let (last_position, last_record) = mrecordlog.last_record("test-queue").unwrap().unwrap(); - assert_eq!(last_position, 0); - assert!(is_eof_mrecord(&last_record)); - - append_eof_record_if_necessary(&mut mrecordlog, &"test-queue".to_string()).await; - let (last_position, last_record) = mrecordlog.last_record("test-queue").unwrap().unwrap(); - assert_eq!(last_position, 0); - assert!(is_eof_mrecord(&last_record)); - - mrecordlog.truncate("test-queue", 0).await.unwrap(); - - append_eof_record_if_necessary(&mut mrecordlog, &"test-queue".to_string()).await; - let (last_position, last_record) = mrecordlog.last_record("test-queue").unwrap().unwrap(); - assert_eq!(last_position, 1); - assert!(is_eof_mrecord(&last_record)); - } - #[tokio::test] async fn test_check_enough_capacity() { let tempdir = tempfile::tempdir().unwrap(); @@ -183,42 +131,31 @@ mod tests { } #[tokio::test] - async fn test_get_truncation_position() { + async fn test_append_queue_position_range() { let tempdir = tempfile::tempdir().unwrap(); let mut mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); - let queue_id = "test-queue".to_string(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id); - assert!(truncation_position.is_none()); + assert!(queue_position_range(&mrecordlog, &"queue-not-found".to_string()).is_none()); - mrecordlog.create_queue(&queue_id).await.unwrap(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id).unwrap(); - assert_eq!(truncation_position, Position::Beginning); + mrecordlog.create_queue("test-queue").await.unwrap(); + assert!(queue_position_range(&mrecordlog, &"test-queue".to_string()).is_none()); mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-record-foo")) - .await - .unwrap(); - mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-record-bar")) + .append_record("test-queue", None, &b"test-doc-foo"[..]) .await .unwrap(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id).unwrap(); - assert_eq!(truncation_position, Position::Beginning); - - mrecordlog.truncate(&queue_id, 0).await.unwrap(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id).unwrap(); - assert_eq!(truncation_position, Position::from(0u64)); - - mrecordlog.truncate(&queue_id, 1).await.unwrap(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id).unwrap(); - assert_eq!(truncation_position, Position::from(1u64)); + let position_range = queue_position_range(&mrecordlog, &"test-queue".to_string()).unwrap(); + assert_eq!(position_range, 0..=0); mrecordlog - .append_record(&queue_id, None, MRecord::Eof.encode()) + .append_record("test-queue", None, &b"test-doc-bar"[..]) .await .unwrap(); - let truncation_position = get_truncation_position(&mrecordlog, &queue_id).unwrap(); - assert_eq!(truncation_position, Position::Eof); + let position_range = queue_position_range(&mrecordlog, &"test-queue".to_string()).unwrap(); + assert_eq!(position_range, 0..=1); + + mrecordlog.truncate("test-queue", 0).await.unwrap(); + let position_range = queue_position_range(&mrecordlog, &"test-queue".to_string()).unwrap(); + assert_eq!(position_range, 1..=1); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 38b32ff746f..87ded772a49 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -610,7 +610,9 @@ impl ReplicationTask { .await .expect("TODO") } - .into(); + .map(Position::offset) + .expect("records should not be empty"); + let batch_num_bytes = doc_batch.num_bytes() as u64; let batch_num_docs = doc_batch.num_docs() as u64; @@ -861,7 +863,7 @@ mod tests { shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(0u64)), + to_position_inclusive: Some(Position::offset(0u64)), }, ReplicateSubrequest { subrequest_id: 1, @@ -870,7 +872,7 @@ mod tests { shard_id: 2, doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(1u64)), + to_position_inclusive: Some(Position::offset(1u64)), }, ReplicateSubrequest { subrequest_id: 2, @@ -878,8 +880,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-qux", "test-doc-tux"])), - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(2u64)), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(2u64)), }, ]; let replicate_response = replication_stream_task_handle @@ -901,19 +903,28 @@ mod tests { 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.replication_position_inclusive(), 0u64); + assert_eq!( + replicate_success_0.replication_position_inclusive(), + Position::offset(0u64) + ); 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.replication_position_inclusive(), 1u64); + assert_eq!( + replicate_success_1.replication_position_inclusive(), + Position::offset(1u64) + ); 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.replication_position_inclusive(), 2u64); + assert_eq!( + replicate_success_2.replication_position_inclusive(), + Position::offset(2u64) + ); } #[tokio::test] @@ -1099,7 +1110,7 @@ mod tests { shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(0u64)), + to_position_inclusive: Some(Position::offset(0u64)), }, ReplicateSubrequest { subrequest_id: 1, @@ -1108,7 +1119,7 @@ mod tests { shard_id: 2, doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(1u64)), + to_position_inclusive: Some(Position::offset(1u64)), }, ReplicateSubrequest { subrequest_id: 2, @@ -1117,7 +1128,7 @@ mod tests { shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-qux", "test-doc-tux"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(1u64)), + to_position_inclusive: Some(Position::offset(1u64)), }, ], replication_seqno: 3, @@ -1140,19 +1151,28 @@ mod tests { 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.replication_position_inclusive(), 0u64); + assert_eq!( + replicate_success_0.replication_position_inclusive(), + Position::offset(0u64) + ); 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.replication_position_inclusive(), 1u64); + assert_eq!( + replicate_success_1.replication_position_inclusive(), + Position::offset(1u64) + ); 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.replication_position_inclusive(), 1u64); + assert_eq!( + replicate_success_2.replication_position_inclusive(), + Position::offset(1u64) + ); let state_guard = state.read().await; @@ -1183,8 +1203,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-moo"])), - from_position_exclusive: Some(Position::from(0u64)), - to_position_inclusive: Some(Position::from(1u64)), + from_position_exclusive: Some(Position::offset(0u64)), + to_position_inclusive: Some(Position::offset(1u64)), }], replication_seqno: 4, }; @@ -1206,7 +1226,10 @@ mod tests { 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.replication_position_inclusive(), 1u64); + assert_eq!( + replicate_success_0.replication_position_inclusive(), + Position::offset(1u64) + ); let state_guard = state.read().await; @@ -1274,8 +1297,8 @@ mod tests { source_id: "test-source".to_string(), shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), - from_position_exclusive: Position::from(0u64).into(), - to_position_inclusive: Some(Position::from(1u64)), + from_position_exclusive: Position::offset(0u64).into(), + to_position_inclusive: Some(Position::offset(1u64)), }], replication_seqno: 0, }; @@ -1360,7 +1383,7 @@ mod tests { shard_id: 1, doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: None, - to_position_inclusive: Some(Position::from(0u64)), + to_position_inclusive: Some(Position::offset(0u64)), }], replication_seqno: 0, }; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 4da0c39faa3..27ba0df4b7f 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -1082,14 +1082,14 @@ mod tests { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - replication_position_inclusive: Some(Position::from(1u64)), + 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, - replication_position_inclusive: Some(Position::from(0u64)), + replication_position_inclusive: Some(Position::offset(0u64)), }, ], failures: Vec::new(), @@ -1122,7 +1122,7 @@ mod tests { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - replication_position_inclusive: Some(Position::from(3u64)), + replication_position_inclusive: Some(Position::offset(3u64)), }], failures: Vec::new(), }; @@ -1158,7 +1158,7 @@ mod tests { index_uid: "test-index-1:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, - replication_position_inclusive: Some(Position::from(0u64)), + replication_position_inclusive: Some(Position::offset(0u64)), }], failures: Vec::new(), }; @@ -1293,7 +1293,7 @@ mod tests { index_uid: "test-index-0:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - replication_position_inclusive: Some(Position::from(0u64)), + replication_position_inclusive: Some(Position::offset(0u64)), }], failures: Vec::new(), }; diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 4b9847a8335..7a4c5dbbd15 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -405,7 +405,7 @@ mod tests { // for the pipeline state to be updated. test_sandbox .universe() - .sleep(OBSERVE_PIPELINE_INTERVAL * 3) + .sleep(OBSERVE_PIPELINE_INTERVAL * 5) .await; let pipeline_state = pipeline_handler.process_pending_and_observe().await.state; assert_eq!(pipeline_state.delete_task_planner.metrics.num_errors, 1); diff --git a/quickwit/quickwit-metastore/src/checkpoint.rs b/quickwit/quickwit-metastore/src/checkpoint.rs index 74fbed6bd9b..c697d82e39a 100644 --- a/quickwit/quickwit-metastore/src/checkpoint.rs +++ b/quickwit/quickwit-metastore/src/checkpoint.rs @@ -196,7 +196,7 @@ impl SourceCheckpoint { /// let checkpoint: SourceCheckpoint = [(0u64, 0u64), (1u64, 2u64)] /// .into_iter() /// .map(|(partition_id, offset)| { -/// (PartitionId::from(partition_id), Position::from(offset)) +/// (PartitionId::from(partition_id), Position::offset(offset)) /// }) /// .collect(); /// ``` @@ -214,7 +214,7 @@ impl Serialize for SourceCheckpoint { where S: serde::Serializer { let mut map = serializer.serialize_map(Some(self.per_partition.len()))?; for (partition, position) in &self.per_partition { - map.serialize_entry(&*partition.0, position.as_str())?; + map.serialize_entry(&*partition.0, position)?; } map.end() } @@ -344,7 +344,7 @@ impl fmt::Debug for SourceCheckpoint { for (i, (partition_id, position)) in self.per_partition.iter().enumerate() { f.write_str(&partition_id.0)?; f.write_str(":")?; - f.write_str(position.as_str())?; + write!(f, "{}", position)?; let is_last = i == self.per_partition.len() - 1; if !is_last { f.write_str(" ")?; @@ -404,9 +404,7 @@ impl fmt::Debug for SourceCheckpointDelta { write!( f, "{}:({}..{}]", - partition_id.0, - partition_delta.from.as_str(), - partition_delta.to.as_str() + partition_id.0, partition_delta.from, partition_delta.to, )?; if i != self.per_partition.len() - 1 { f.write_str(" ")?; @@ -426,12 +424,12 @@ impl TryFrom> for SourceCheckpointDelta { let from_position = if range.start == 0 { Position::Beginning } else { - Position::from(range.start - 1) + Position::offset(range.start - 1) }; let to_position = if range.end == 0 { Position::Beginning } else { - Position::from(range.end - 1) + Position::offset(range.end - 1) }; SourceCheckpointDelta::from_partition_delta( PartitionId::default(), @@ -565,15 +563,15 @@ mod tests { let delta = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from(123u64), - Position::from(128u64), + Position::offset(123u64), + Position::offset(128u64), ) .unwrap(); delta .record_partition_delta( PartitionId::from("b"), - Position::from(60002u64), - Position::from(60187u64), + Position::offset(60002u64), + Position::offset(60187u64), ) .unwrap(); delta @@ -597,14 +595,14 @@ mod tests { let delta1 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00123"), - Position::from("00128"), + Position::offset("00123"), + Position::offset("00128"), ) .unwrap(); delta.record_partition_delta( PartitionId::from("b"), - Position::from("60002"), - Position::from("60187"), + Position::offset("60002"), + Position::offset("60187"), )?; delta }; @@ -612,14 +610,14 @@ mod tests { let delta2 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00128"), - Position::from("00129"), + Position::offset("00128"), + Position::offset("00129"), ) .unwrap(); delta.record_partition_delta( PartitionId::from("b"), - Position::from("50099"), - Position::from("60002"), + Position::offset("50099"), + Position::offset("60002"), )?; delta }; @@ -638,14 +636,14 @@ mod tests { let delta1 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00123"), - Position::from("00128"), + Position::offset("00123"), + Position::offset("00128"), ) .unwrap(); delta.record_partition_delta( PartitionId::from("b"), - Position::from("60002"), - Position::from("60187"), + Position::offset("60002"), + Position::offset("60187"), )?; delta }; @@ -653,14 +651,14 @@ mod tests { let delta3 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("b"), - Position::from("60187"), - Position::from("60190"), + Position::offset("60187"), + Position::offset("60190"), ) .unwrap(); delta.record_partition_delta( PartitionId::from("c"), - Position::from("20001"), - Position::from("20008"), + Position::offset("20001"), + Position::offset("20008"), )?; delta }; @@ -674,15 +672,15 @@ mod tests { let mut delta1 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00123"), - Position::from("00128"), + Position::offset("00123"), + Position::offset("00128"), ) .unwrap(); delta .record_partition_delta( PartitionId::from("b"), - Position::from("60002"), - Position::from("60187"), + Position::offset("60002"), + Position::offset("60187"), ) .unwrap(); delta @@ -690,15 +688,15 @@ mod tests { let delta2 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("b"), - Position::from("60187"), - Position::from("60348"), + Position::offset("60187"), + Position::offset("60348"), ) .unwrap(); delta .record_partition_delta( PartitionId::from("c"), - Position::from("20001"), - Position::from("20008"), + Position::offset("20001"), + Position::offset("20008"), ) .unwrap(); delta @@ -706,22 +704,22 @@ mod tests { let delta3 = { let mut delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00123"), - Position::from("00128"), + Position::offset("00123"), + Position::offset("00128"), ) .unwrap(); delta .record_partition_delta( PartitionId::from("b"), - Position::from("60002"), - Position::from("60348"), + Position::offset("60002"), + Position::offset("60348"), ) .unwrap(); delta .record_partition_delta( PartitionId::from("c"), - Position::from("20001"), - Position::from("20008"), + Position::offset("20001"), + Position::offset("20008"), ) .unwrap(); delta @@ -731,8 +729,8 @@ mod tests { let delta4 = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("00130"), - Position::from("00142"), + Position::offset("00130"), + Position::offset("00142"), ) .unwrap(); let result = delta1.extend(delta4); @@ -740,25 +738,19 @@ mod tests { result, Err(PartitionDeltaError::from(IncompatibleCheckpointDelta { partition_id: PartitionId::from("a"), - partition_position: Position::from("00128"), - delta_from_position: Position::from("00130") + partition_position: Position::offset("00128"), + delta_from_position: Position::offset("00130") })) ); } - #[test] - fn test_position_u64() { - let pos = Position::from(4u64); - assert_eq!(pos.as_str(), "00000000000000000004"); - } - #[test] fn test_record_negative_partition_delta_is_failing() { { let delta_error = SourceCheckpointDelta::from_partition_delta( PartitionId::from("a"), - Position::from("20"), - Position::from("20"), + Position::offset("20"), + Position::offset("20"), ) .unwrap_err(); matches!( @@ -771,8 +763,8 @@ mod tests { let delta_error = delta .record_partition_delta( PartitionId::from("a"), - Position::from("20"), - Position::from("10"), + Position::offset("20"), + Position::offset("10"), ) .unwrap_err(); matches!( @@ -810,14 +802,14 @@ mod tests { let partition = PartitionId::from("a"); let delta = SourceCheckpointDelta::from_partition_delta( partition.clone(), - Position::from(42u64), - Position::from(43u64), + Position::offset(42u64), + Position::offset(43u64), ) .unwrap(); let checkpoint: SourceCheckpoint = delta.get_source_checkpoint(); assert_eq!( checkpoint.position_for_partition(&partition).unwrap(), - &Position::from(43u64) + &Position::offset(43u64) ); } } 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 d2786b67008..88ea44122c4 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 @@ -215,7 +215,7 @@ impl Shards { for shard_id in subrequest.shard_ids { if let Entry::Occupied(entry) = self.shards.entry(shard_id) { let shard = entry.get(); - if !force && shard.publish_position_inclusive() != Position::Eof { + if !force && !shard.publish_position_inclusive().is_eof() { let message = format!("shard `{shard_id}` is not deletable"); return Err(MetastoreError::InvalidArgument { message }); } @@ -284,7 +284,7 @@ impl Shards { for (shard_id, publish_position_inclusive) in shard_ids { let shard = self.get_shard_mut(shard_id).expect("shard should exist"); - if publish_position_inclusive == Position::Eof { + if publish_position_inclusive.is_eof() { shard.shard_state = ShardState::Closed as i32; } shard.publish_position_inclusive = Some(publish_position_inclusive); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index 24129bec968..775aea646b6 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -145,7 +145,7 @@ impl TestableForRegression for IndexMetadata { let delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from(0i64), Position::Beginning, - Position::from(42u64), + Position::offset(42u64), ) .unwrap(); source_checkpoint.try_apply_delta(delta).unwrap(); diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index ad641af731e..64033d632c1 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -943,7 +943,7 @@ pub async fn test_metastore_publish_splits_empty_splits_array_is_allowed< source_checkpoint .position_for_partition(&PartitionId::default()) .unwrap(), - &Position::from(100u64 - 1) + &Position::offset(100u64 - 1) ); cleanup_index(&mut metastore, index_uid).await; } @@ -1565,7 +1565,7 @@ pub async fn test_metastore_publish_splits_concurrency< let source_delta = SourceCheckpointDelta::from_partition_delta( PartitionId::from(partition_id as u64), Position::Beginning, - Position::from(partition_id as u64), + Position::offset(partition_id as u64), ) .unwrap(); let checkpoint_delta = IndexCheckpointDelta { diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index 1d0c8eec430..fe21cc61c48 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -32,7 +32,7 @@ service IngesterService { // Streams records from a leader or a follower. The client can optionally specify a range of positions to fetch, // otherwise the stream will go undefinitely or until the shard is closed. - rpc OpenFetchStream(OpenFetchStreamRequest) returns (stream FetchResponseV2); + rpc OpenFetchStream(OpenFetchStreamRequest) returns (stream FetchMessage); // Streams status updates, called "observations", from an ingester. rpc OpenObservationStream(OpenObservationStreamRequest) returns (stream ObservationMessage); @@ -211,7 +211,14 @@ message OpenFetchStreamRequest { quickwit.ingest.Position from_position_exclusive = 5; } -message FetchResponseV2 { +message FetchMessage { + oneof message { + FetchPayload payload = 1; + FetchEof eof = 2; + } +} + +message FetchPayload { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; @@ -220,6 +227,13 @@ message FetchResponseV2 { quickwit.ingest.Position to_position_inclusive = 6; } +message FetchEof { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + quickwit.ingest.Position eof_position = 4; +} + message InitShardsRequest { repeated quickwit.ingest.Shard shards = 1; } 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 6db5fb9b389..7570019c256 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -271,7 +271,27 @@ pub struct OpenFetchStreamRequest { #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct FetchResponseV2 { +pub struct FetchMessage { + #[prost(oneof = "fetch_message::Message", tags = "1, 2")] + pub message: ::core::option::Option, +} +/// Nested message and enum types in `FetchMessage`. +pub mod fetch_message { + #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] + #[serde(rename_all = "snake_case")] + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Message { + #[prost(message, tag = "1")] + Payload(super::FetchPayload), + #[prost(message, tag = "2")] + Eof(super::FetchEof), + } +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct FetchPayload { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, #[prost(string, tag = "2")] @@ -288,6 +308,19 @@ pub struct FetchResponseV2 { #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct FetchEof { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(uint64, tag = "3")] + pub shard_id: u64, + #[prost(message, optional, tag = "4")] + pub eof_position: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct InitShardsRequest { #[prost(message, repeated, tag = "1")] pub shards: ::prost::alloc::vec::Vec, @@ -489,7 +522,7 @@ pub trait IngesterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + async fn open_fetch_stream( &mut self, request: OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result>; + ) -> crate::ingest::IngestV2Result>; /// Streams status updates, called "observations", from an ingester. async fn open_observation_stream( &mut self, @@ -615,7 +648,7 @@ impl IngesterService for IngesterServiceClient { async fn open_fetch_stream( &mut self, request: OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result> { + ) -> crate::ingest::IngestV2Result> { self.inner.open_fetch_stream(request).await } async fn open_observation_stream( @@ -681,9 +714,7 @@ pub mod ingester_service_mock { async fn open_fetch_stream( &mut self, request: super::OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result< - IngesterServiceStream, - > { + ) -> crate::ingest::IngestV2Result> { self.inner.lock().await.open_fetch_stream(request).await } async fn open_observation_stream( @@ -774,7 +805,7 @@ for Box { } } impl tower::Service for Box { - type Response = IngesterServiceStream; + type Response = IngesterServiceStream; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; fn poll_ready( @@ -901,7 +932,7 @@ struct IngesterServiceTowerBlock { >, open_fetch_stream_svc: quickwit_common::tower::BoxService< OpenFetchStreamRequest, - IngesterServiceStream, + IngesterServiceStream, crate::ingest::IngestV2Error, >, open_observation_stream_svc: quickwit_common::tower::BoxService< @@ -968,7 +999,7 @@ impl IngesterService for IngesterServiceTowerBlock { async fn open_fetch_stream( &mut self, request: OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result> { + ) -> crate::ingest::IngestV2Result> { self.open_fetch_stream_svc.ready().await?.call(request).await } async fn open_observation_stream( @@ -1033,7 +1064,7 @@ pub struct IngesterServiceTowerBlockBuilder { quickwit_common::tower::BoxLayer< Box, OpenFetchStreamRequest, - IngesterServiceStream, + IngesterServiceStream, crate::ingest::IngestV2Error, >, >, @@ -1112,7 +1143,7 @@ impl IngesterServiceTowerBlockBuilder { >>::Future: Send + 'static, L::Service: tower::Service< OpenFetchStreamRequest, - Response = IngesterServiceStream, + Response = IngesterServiceStream, Error = crate::ingest::IngestV2Error, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, @@ -1220,7 +1251,7 @@ impl IngesterServiceTowerBlockBuilder { L: tower::Layer> + Send + Sync + 'static, L::Service: tower::Service< OpenFetchStreamRequest, - Response = IngesterServiceStream, + Response = IngesterServiceStream, Error = crate::ingest::IngestV2Error, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, @@ -1504,10 +1535,10 @@ where > + tower::Service< OpenFetchStreamRequest, - Response = IngesterServiceStream, + Response = IngesterServiceStream, Error = crate::ingest::IngestV2Error, Future = BoxFuture< - IngesterServiceStream, + IngesterServiceStream, crate::ingest::IngestV2Error, >, > @@ -1566,7 +1597,7 @@ where async fn open_fetch_stream( &mut self, request: OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result> { + ) -> crate::ingest::IngestV2Result> { self.call(request).await } async fn open_observation_stream( @@ -1667,7 +1698,7 @@ where async fn open_fetch_stream( &mut self, request: OpenFetchStreamRequest, - ) -> crate::ingest::IngestV2Result> { + ) -> crate::ingest::IngestV2Result> { self.inner .open_fetch_stream(request) .await @@ -1787,7 +1818,7 @@ for IngesterServiceGrpcServerAdapter { .map_err(|error| error.into()) } type OpenFetchStreamStream = quickwit_common::ServiceStream< - tonic::Result, + tonic::Result, >; async fn open_fetch_stream( &self, @@ -2025,7 +2056,7 @@ pub mod ingester_service_grpc_client { &mut self, request: impl tonic::IntoRequest, ) -> std::result::Result< - tonic::Response>, + tonic::Response>, tonic::Status, > { self.inner @@ -2261,7 +2292,7 @@ pub mod ingester_service_grpc_server { >; /// Server streaming response type for the OpenFetchStream method. type OpenFetchStreamStream: futures_core::Stream< - Item = std::result::Result, + Item = std::result::Result, > + Send + 'static; @@ -2506,7 +2537,7 @@ pub mod ingester_service_grpc_server { > tonic::server::ServerStreamingService< super::OpenFetchStreamRequest, > for OpenFetchStreamSvc { - type Response = super::FetchResponseV2; + type Response = super::FetchMessage; type ResponseStream = T::OpenFetchStreamStream; type Future = BoxFuture< tonic::Response, diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs index b743124f77a..cede71bf984 100644 --- a/quickwit/quickwit-proto/src/ingest/ingester.rs +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -23,7 +23,31 @@ include!("../codegen/quickwit/quickwit.ingest.ingester.rs"); pub use ingester_service_grpc_server::IngesterServiceGrpcServer; -impl FetchResponseV2 { +impl FetchMessage { + pub fn new_payload(payload: FetchPayload) -> Self { + assert!( + matches!(&payload.mrecord_batch, Some(batch) if !batch.mrecord_lengths.is_empty()), + "`mrecord_batch` must be set and non-empty" + ); + + Self { + message: Some(fetch_message::Message::Payload(payload)), + } + } + + pub fn new_eof(eof: FetchEof) -> Self { + assert!( + matches!(eof.eof_position, Some(Position::Eof(_))), + "`eof_position` must be set" + ); + + Self { + message: Some(fetch_message::Message::Eof(eof)), + } + } +} + +impl FetchPayload { pub fn queue_id(&self) -> QueueId { queue_id(&self.index_uid, &self.source_id, self.shard_id) } @@ -35,6 +59,20 @@ impl FetchResponseV2 { 0 } } + + pub fn from_position_exclusive(&self) -> Position { + self.from_position_exclusive.clone().unwrap_or_default() + } + + pub fn to_position_inclusive(&self) -> Position { + self.to_position_inclusive.clone().unwrap_or_default() + } +} + +impl FetchEof { + pub fn eof_position(&self) -> Position { + self.eof_position.clone().unwrap_or_default() + } } impl OpenFetchStreamRequest { diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index 8636ecc3bc2..fc42e05b1fe 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -19,7 +19,7 @@ use bytes::Bytes; -use self::ingester::{FetchResponseV2, PersistFailureReason, ReplicateFailureReason}; +use self::ingester::{PersistFailureReason, ReplicateFailureReason}; use self::router::IngestFailureReason; use super::types::NodeId; use super::{ServiceError, ServiceErrorCode}; @@ -123,16 +123,6 @@ impl DocBatchV2 { } } -impl FetchResponseV2 { - pub fn from_position_exclusive(&self) -> Position { - self.from_position_exclusive.clone().unwrap_or_default() - } - - pub fn to_position_inclusive(&self) -> Position { - self.to_position_inclusive.clone().unwrap_or_default() - } -} - impl MRecordBatch { pub fn encoded_mrecords(&self) -> impl Iterator + '_ { self.mrecord_lengths @@ -156,6 +146,21 @@ impl MRecordBatch { pub fn num_mrecords(&self) -> usize { self.mrecord_lengths.len() } + + #[cfg(any(test, feature = "testsuite"))] + pub fn for_test(mrecords: impl IntoIterator) -> Option { + let mut mrecord_buffer = Vec::new(); + let mut mrecord_lengths = Vec::new(); + + for mrecord in mrecords { + mrecord_buffer.extend(mrecord.as_bytes()); + mrecord_lengths.push(mrecord.len() as u32); + } + Some(Self { + mrecord_lengths, + mrecord_buffer: Bytes::from(mrecord_buffer), + }) + } } impl Shard { diff --git a/quickwit/quickwit-proto/src/types/position.rs b/quickwit/quickwit-proto/src/types/position.rs index d3321cb0ddd..4a76687043e 100644 --- a/quickwit/quickwit-proto/src/types/position.rs +++ b/quickwit/quickwit-proto/src/types/position.rs @@ -20,14 +20,14 @@ use std::fmt; use std::fmt::{Debug, Display}; -use bytes::Bytes; +use bytes::{Bytes, BytesMut}; use bytestring::ByteString; use prost::{self, DecodeError}; use serde::{Deserialize, Serialize}; const BEGINNING: &str = ""; -const EOF: &str = "~eof"; +const EOF_PREFIX: &str = "~"; #[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Ord, PartialOrd)] pub struct Offset(ByteString); @@ -56,6 +56,30 @@ impl fmt::Display for Offset { } } +impl From for Offset { + fn from(offset: i64) -> Self { + Self(ByteString::from(format!("{offset:0>20}"))) + } +} + +impl From for Offset { + fn from(offset: u64) -> Self { + Self(ByteString::from(format!("{offset:0>20}"))) + } +} + +impl From for Offset { + fn from(offset: usize) -> Self { + Self(ByteString::from(format!("{offset:0>20}"))) + } +} + +impl From<&str> for Offset { + fn from(offset: &str) -> Self { + Self(ByteString::from(offset)) + } +} + /// Marks a position within a specific partition/shard of a source. /// /// The nature of the position depends on the source. @@ -74,162 +98,120 @@ pub enum Position { #[default] Beginning, Offset(Offset), - Eof, + /// End of partition/shard at the given offset. `Eof(None)` means no records were ever written. + Eof(Option), } impl Debug for Position { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { match self { Self::Beginning => write!(f, "Position::Beginning"), - // The derive implementation would show `Offset(Offset(0000001))` here. - Self::Offset(offset) => write!(f, "Position::Offset({})", offset.0), - Self::Eof => write!(f, "Position::Eof"), + // The derive implementation would show `Offset(Offset(0000001u64))` here. + Self::Offset(offset) => write!(f, "Position::Offset({offset})"), + Self::Eof(Some(offset)) => write!(f, "Position::Eof({offset})"), + Self::Eof(None) => write!(f, "Position::Eof"), } } } impl Position { - fn as_bytes(&self) -> Bytes { - match self { - Self::Beginning => Bytes::from_static(BEGINNING.as_bytes()), - Self::Offset(offset) => offset.0.as_bytes().clone(), - Self::Eof => Bytes::from_static(EOF.as_bytes()), - } + pub fn offset(offset: impl Into) -> Self { + Self::Offset(offset.into()) } - pub fn as_str(&self) -> &str { + pub fn eof(offset: impl Into) -> Self { + Self::Eof(Some(offset.into())) + } + + pub fn as_eof(&self) -> Self { match self { - Self::Beginning => BEGINNING, - Self::Offset(offset) => offset.as_str(), - Self::Eof => EOF, + Self::Beginning => Self::Eof(None), + Self::Offset(offset) => Self::Eof(Some(offset.clone())), + _ => self.clone(), } } pub fn as_i64(&self) -> Option { match self { - Self::Beginning => None, - Self::Offset(offset) => offset.as_i64(), - Self::Eof => None, + Self::Offset(offset) | Self::Eof(Some(offset)) => offset.as_i64(), + _ => None, } } pub fn as_u64(&self) -> Option { match self { - Self::Beginning => None, - Self::Offset(offset) => offset.as_u64(), - Self::Eof => None, + Self::Offset(offset) | Self::Eof(Some(offset)) => offset.as_u64(), + _ => None, } } pub fn as_usize(&self) -> Option { match self { - Self::Beginning => None, - Self::Offset(offset) => offset.as_usize(), - Self::Eof => None, + Self::Offset(offset) | Self::Eof(Some(offset)) => offset.as_usize(), + _ => None, } } -} - -impl Display for Position { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", self.as_str()) - } -} - -impl From for Position { - fn from(offset: i64) -> Self { - assert!(offset >= 0); - Self::from(format!("{offset:0>20}")) - } -} - -impl From for Position { - fn from(offset: u64) -> Self { - Self::from(format!("{offset:0>20}")) - } -} -impl From for Position { - fn from(offset: usize) -> Self { - Self::from(format!("{offset:0>20}")) + pub fn is_eof(&self) -> bool { + matches!(self, Self::Eof(_)) } -} -impl From> for Position -where Position: From -{ - fn from(offset_opt: Option) -> Self { - match offset_opt { - Some(offset) => Self::from(offset), - None => Self::Beginning, + fn as_bytes(&self) -> Bytes { + match self { + Self::Beginning => Bytes::from_static(BEGINNING.as_bytes()), + Self::Offset(offset) => offset.0.as_bytes().clone(), + Self::Eof(Some(offset)) => { + let mut bytes = BytesMut::with_capacity(EOF_PREFIX.len() + offset.0.len()); + bytes.extend_from_slice(EOF_PREFIX.as_bytes()); + bytes.extend_from_slice(offset.0.as_bytes()); + bytes.freeze() + } + Self::Eof(None) => Bytes::from_static(EOF_PREFIX.as_bytes()), } } } -impl PartialEq for Position { - fn eq(&self, other: &i64) -> bool { - self.as_i64() == Some(*other) - } -} - -impl PartialEq for Position { - fn eq(&self, other: &u64) -> bool { - self.as_u64() == Some(*other) - } -} - -impl PartialEq for Position { - fn eq(&self, other: &usize) -> bool { - self.as_usize() == Some(*other) - } -} - -impl From for Position { - fn from(string: String) -> Self { - Self::from(ByteString::from(string)) - } -} - -impl From<&'static str> for Position { - fn from(string: &'static str) -> Self { - Self::from(ByteString::from_static(string)) +impl Display for Position { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + Self::Beginning => write!(f, "{BEGINNING}"), + Self::Offset(offset) => write!(f, "{offset}"), + Self::Eof(Some(offset)) => write!(f, "{EOF_PREFIX}{offset}"), + Self::Eof(None) => write!(f, "{EOF_PREFIX}"), + } } } impl From for Position { - fn from(byte_string: ByteString) -> Self { - if byte_string.is_empty() { - Self::Beginning - } else if byte_string == EOF { - Self::Eof - } else { - Self::Offset(Offset(byte_string)) + fn from(position: ByteString) -> Self { + match &position[..] { + BEGINNING => Self::Beginning, + EOF_PREFIX => Self::Eof(None), + offset if offset.starts_with(EOF_PREFIX) => { + let offset = ByteString::from(&offset[EOF_PREFIX.len()..]); + Self::Eof(Some(Offset(offset))) + } + _ => Self::Offset(Offset(position)), } } } -impl PartialEq<&str> for Position { - fn eq(&self, other: &&str) -> bool { - self.as_str() == *other - } -} - -impl PartialEq for Position { - fn eq(&self, other: &String) -> bool { - self.as_str() == *other +impl From for Position { + fn from(position: String) -> Self { + Self::from(ByteString::from(position)) } } impl Serialize for Position { fn serialize(&self, serializer: S) -> Result { - serializer.serialize_str(self.as_str()) + serializer.collect_str(self) } } impl<'de> Deserialize<'de> for Position { fn deserialize>(deserializer: D) -> Result { - let string = String::deserialize(deserializer)?; - Ok(Self::from(string)) + let position_str = String::deserialize(deserializer)?; + Ok(Self::from(position_str)) } } @@ -286,21 +268,29 @@ mod tests { use super::*; #[test] - fn test_position_partial_eq() { - assert_eq!(Position::Beginning, ""); - assert_eq!(Position::Beginning, "".to_string()); - assert_eq!(Position::from(0u64), 0i64); - assert_eq!(Position::from(0u64), 0u64); - assert_eq!(Position::from(0u64), 0usize); + fn test_position_eof() { + let eof_position = Position::Beginning.as_eof(); + + assert!(eof_position.is_eof()); + assert!(eof_position.as_u64().is_none()); + + let eof_position = Position::offset(0u64).as_eof(); + + assert!(eof_position.is_eof()); + assert_eq!(eof_position.as_u64().unwrap(), 0u64); } #[test] #[allow(clippy::cmp_owned)] fn test_position_ord() { - assert!(Position::Beginning < Position::from(0u64)); - assert!(Position::from(0u64) < Position::from(1u64)); - assert!(Position::from(1u64) < Position::Eof); - assert!(Position::Beginning < Position::Eof); + assert!(Position::Beginning < Position::offset(0u64)); + assert!(Position::Beginning < Position::Eof(None)); + assert!(Position::Beginning < Position::eof(0u64)); + + assert!(Position::offset(0u64) < Position::offset(1u64)); + + assert!(Position::Eof(None) < Position::eof(0u64)); + assert!(Position::eof(0u64) < Position::eof(1u64)); } #[test] @@ -310,45 +300,66 @@ mod tests { let deserialized: Position = serde_json::from_str(&serialized).unwrap(); assert_eq!(deserialized, Position::Beginning); - let serialized = serde_json::to_string(&Position::from(0u64)).unwrap(); + let serialized = serde_json::to_string(&Position::offset(0u64)).unwrap(); assert_eq!(serialized, r#""00000000000000000000""#); let deserialized: Position = serde_json::from_str(&serialized).unwrap(); - assert_eq!(deserialized, Position::from(0u64)); + assert_eq!(deserialized, Position::offset(0u64)); + + let serialized = serde_json::to_string(&Position::Eof(None)).unwrap(); + assert_eq!(serialized, r#""~""#); + let deserialized: Position = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, Position::Eof(None)); - let serialized = serde_json::to_string(&Position::Eof).unwrap(); - assert_eq!(serialized, r#""~eof""#); + let serialized = serde_json::to_string(&Position::eof(0u64)).unwrap(); + assert_eq!(serialized, r#""~00000000000000000000""#); let deserialized: Position = serde_json::from_str(&serialized).unwrap(); - assert_eq!(deserialized, Position::Eof); + assert_eq!(deserialized, Position::eof(0u64)); } #[test] fn test_position_prost_serde_roundtrip() { - let mut buffer = Vec::new(); - Position::Beginning.encode(&mut buffer).unwrap(); + let encoded = Position::Beginning.encode_to_vec(); assert_eq!( - Position::decode(Bytes::from(buffer)).unwrap(), + Position::decode(Bytes::from(encoded)).unwrap(), + Position::Beginning + ); + let encoded = Position::Beginning.encode_length_delimited_to_vec(); + assert_eq!( + Position::decode_length_delimited(Bytes::from(encoded)).unwrap(), Position::Beginning ); - let mut buffer = Vec::new(); - Position::from("0").encode(&mut buffer).unwrap(); + let encoded = Position::offset(0u64).encode_to_vec(); + assert_eq!( + Position::decode(Bytes::from(encoded)).unwrap(), + Position::offset(0u64) + ); + let encoded = Position::offset(0u64).encode_length_delimited_to_vec(); assert_eq!( - Position::decode(Bytes::from(buffer)).unwrap(), - Position::from("0") + Position::decode_length_delimited(Bytes::from(encoded)).unwrap(), + Position::offset(0u64) ); - let mut buffer = Vec::new(); - Position::from(0u64).encode(&mut buffer).unwrap(); + let encoded = Position::Eof(None).encode_to_vec(); + assert_eq!( + Position::decode(Bytes::from(encoded)).unwrap(), + Position::Eof(None) + ); + let encoded = Position::Eof(None).encode_length_delimited_to_vec(); assert_eq!( - Position::decode(Bytes::from(buffer)).unwrap(), - Position::from(0u64) + Position::decode_length_delimited(Bytes::from(encoded)).unwrap(), + Position::Eof(None) ); - let mut buffer = Vec::new(); - Position::Eof.encode(&mut buffer).unwrap(); + let encoded = Position::eof(0u64).encode_to_vec(); + assert_eq!( + Position::decode(Bytes::from(encoded)).unwrap(), + Position::eof(0u64) + ); + let encoded = Position::eof(0u64).encode_length_delimited_to_vec(); assert_eq!( - Position::decode(Bytes::from(buffer)).unwrap(), - Position::Eof + Position::decode_length_delimited(Bytes::from(encoded)).unwrap(), + Position::eof(0u64) ); } }