From fdc60f81689a211993bfb294f0fd3b3550d08bb6 Mon Sep 17 00:00:00 2001 From: Adrien Guillo Date: Wed, 25 Oct 2023 07:34:02 -0400 Subject: [PATCH] Close shards with EOF record (#4021) --- quickwit/Cargo.lock | 10 + quickwit/Cargo.toml | 1 + quickwit/quickwit-cli/src/source.rs | 3 +- .../src/control_plane.rs | 52 +- .../src/control_plane_model.rs | 8 +- .../src/ingest/ingest_controller.rs | 3 +- quickwit/quickwit-control-plane/src/lib.rs | 42 +- .../quickwit-indexing/src/actors/publisher.rs | 4 +- .../quickwit-indexing/src/actors/uploader.rs | 2 +- .../src/models/indexed_split.rs | 2 +- .../src/models/packaged_split.rs | 2 +- .../src/models/publisher_message.rs | 2 +- .../src/source/file_source.rs | 16 +- .../src/source/gcp_pubsub_source.rs | 3 +- .../src/source/ingest/mod.rs | 119 ++- .../src/source/ingest_api_source.rs | 17 +- .../src/source/kafka_source.rs | 15 +- .../src/source/kinesis/kinesis_source.rs | 10 +- .../src/source/pulsar_source.rs | 6 +- .../src/source/vec_source.rs | 26 +- .../quickwit-ingest/src/ingest_v2/fetch.rs | 790 +++++++++------ quickwit/quickwit-ingest/src/ingest_v2/gc.rs | 208 ---- .../src/ingest_v2/ingest_metastore.rs | 43 - .../quickwit-ingest/src/ingest_v2/ingester.rs | 898 +++++++----------- quickwit/quickwit-ingest/src/ingest_v2/mod.rs | 5 +- .../quickwit-ingest/src/ingest_v2/models.rs | 332 +++---- .../quickwit-ingest/src/ingest_v2/mrecord.rs | 36 +- .../src/ingest_v2/replication.rs | 227 ++--- .../quickwit-ingest/src/ingest_v2/router.rs | 5 +- .../src/ingest_v2/shard_table.rs | 3 +- .../src/ingest_v2/test_utils.rs | 162 ++-- .../src/actors/delete_task_pipeline.rs | 4 +- quickwit/quickwit-metastore/src/checkpoint.rs | 85 +- .../src/metastore/control_plane_metastore.rs | 28 +- .../file_backed_index/mod.rs | 39 +- .../file_backed_index/shards.rs | 94 +- .../metastore/file_backed_metastore/mod.rs | 34 +- .../src/metastore/index_metadata/mod.rs | 6 +- .../src/metastore/postgresql_metastore.rs | 28 +- quickwit/quickwit-metastore/src/tests.rs | 4 +- quickwit/quickwit-proto/Cargo.toml | 1 + quickwit/quickwit-proto/build.rs | 9 +- .../protos/quickwit/control_plane.proto | 10 - .../protos/quickwit/ingest.proto | 18 +- .../protos/quickwit/ingester.proto | 19 +- .../protos/quickwit/metastore.proto | 41 - .../protos/quickwit/router.proto | 3 +- .../quickwit/quickwit.control_plane.rs | 429 +-------- .../quickwit/quickwit.ingest.ingester.rs | 34 +- .../quickwit/quickwit.ingest.router.rs | 5 +- .../src/codegen/quickwit/quickwit.ingest.rs | 21 +- .../codegen/quickwit/quickwit.metastore.rs | 291 ------ quickwit/quickwit-proto/src/indexing/mod.rs | 3 +- .../quickwit-proto/src/ingest/ingester.rs | 96 +- quickwit/quickwit-proto/src/ingest/mod.rs | 71 +- .../quickwit-proto/src/metastore/events.rs | 12 +- quickwit/quickwit-proto/src/metastore/mod.rs | 23 +- .../src/{types.rs => types/mod.rs} | 4 + quickwit/quickwit-proto/src/types/position.rs | 327 +++++++ .../quickwit-serve/src/ingest_metastore.rs | 65 -- quickwit/quickwit-serve/src/lib.rs | 50 +- 61 files changed, 1771 insertions(+), 3135 deletions(-) delete mode 100644 quickwit/quickwit-ingest/src/ingest_v2/gc.rs delete mode 100644 quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs rename quickwit/quickwit-proto/src/{types.rs => types/mod.rs} (99%) create mode 100644 quickwit/quickwit-proto/src/types/position.rs delete mode 100644 quickwit/quickwit-serve/src/ingest_metastore.rs diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 2f052c4b08d..ec25546b0a0 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -1093,6 +1093,15 @@ dependencies = [ "either", ] +[[package]] +name = "bytestring" +version = "1.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "238e4886760d98c4f899360c834fa93e62cf7f721ac3c2da375cbdf4b8679aae" +dependencies = [ + "bytes", +] + [[package]] name = "bzip2" version = "0.4.4" @@ -5562,6 +5571,7 @@ dependencies = [ "anyhow", "async-trait", "bytes", + "bytestring", "dyn-clone", "futures", "glob", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index caa691bec8b..cee159ab6b3 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -48,6 +48,7 @@ backoff = { version = "0.4", features = ["tokio"] } base64 = "0.21" byte-unit = { version = "4", default-features = false, features = ["serde", "std"] } bytes = { version = "1", features = ["serde"] } +bytestring = "1.3.0" chitchat = { git = "https://github.com/quickwit-oss/chitchat", rev = "bc29598" } chrono = { version = "0.4.23", default-features = false, features = ["clock", "std"] } clap = { version = "4.4.1", features = ["env", "string"] } diff --git a/quickwit/quickwit-cli/src/source.rs b/quickwit/quickwit-cli/src/source.rs index 353bea71878..2abfd95a1fa 100644 --- a/quickwit/quickwit-cli/src/source.rs +++ b/quickwit/quickwit-cli/src/source.rs @@ -553,7 +553,8 @@ mod tests { use std::str::FromStr; use quickwit_config::{SourceInputFormat, SourceParams}; - use quickwit_metastore::checkpoint::{PartitionId, Position}; + use quickwit_metastore::checkpoint::PartitionId; + use quickwit_proto::types::Position; use serde_json::json; use super::*; diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 824b18bf5a2..3db85b354c9 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -32,11 +32,11 @@ use quickwit_proto::control_plane::{ GetOrCreateOpenShardsResponse, }; use quickwit_proto::metastore::{ - serde_utils as metastore_serde_utils, AddSourceRequest, CloseShardsRequest, CreateIndexRequest, - CreateIndexResponse, DeleteIndexRequest, DeleteShardsRequest, DeleteSourceRequest, - EmptyResponse, MetastoreError, MetastoreService, MetastoreServiceClient, ToggleSourceRequest, + serde_utils as metastore_serde_utils, AddSourceRequest, CreateIndexRequest, + CreateIndexResponse, DeleteIndexRequest, DeleteSourceRequest, EmptyResponse, MetastoreError, + MetastoreService, MetastoreServiceClient, ToggleSourceRequest, }; -use quickwit_proto::{IndexUid, NodeId}; +use quickwit_proto::types::{IndexUid, NodeId}; use serde::Serialize; use tracing::error; @@ -365,50 +365,6 @@ impl Handler for ControlPlane { } } -// This is a metastore callback. Ingesters call the metastore to close shards directly, then the -// metastore notifies the control plane of the event. -#[async_trait] -impl Handler for ControlPlane { - type Reply = ControlPlaneResult; - - async fn handle( - &mut self, - request: CloseShardsRequest, - _ctx: &ActorContext, - ) -> Result { - for close_shards_subrequest in request.subrequests { - let index_uid: IndexUid = close_shards_subrequest.index_uid.into(); - let source_id = close_shards_subrequest.source_id; - // TODO: Group by (index_uid, source_id) first, or change schema of - // `CloseShardsSubrequest`. - let shard_ids = [close_shards_subrequest.shard_id]; - self.model.close_shards(&index_uid, &source_id, &shard_ids) - } - Ok(Ok(EmptyResponse {})) - } -} - -// This is a metastore callback. Ingesters call the metastore to delete shards directly, then the -// metastore notifies the control plane of the event. -#[async_trait] -impl Handler for ControlPlane { - type Reply = ControlPlaneResult; - - async fn handle( - &mut self, - request: DeleteShardsRequest, - _ctx: &ActorContext, - ) -> Result { - for delete_shards_subrequest in request.subrequests { - let index_uid: IndexUid = delete_shards_subrequest.index_uid.into(); - let source_id = delete_shards_subrequest.source_id; - let shard_ids = delete_shards_subrequest.shard_ids; - self.model.delete_shards(&index_uid, &source_id, &shard_ids) - } - Ok(Ok(EmptyResponse {})) - } -} - #[cfg(test)] mod tests { use quickwit_actors::{AskError, Observe, SupervisorMetrics}; diff --git a/quickwit/quickwit-control-plane/src/control_plane_model.rs b/quickwit/quickwit-control-plane/src/control_plane_model.rs index 60f10f92dbd..f5a89fdadcc 100644 --- a/quickwit/quickwit-control-plane/src/control_plane_model.rs +++ b/quickwit/quickwit-control-plane/src/control_plane_model.rs @@ -34,8 +34,7 @@ use quickwit_proto::metastore::{ EntityKind, ListIndexesMetadataRequest, ListShardsSubrequest, MetastoreError, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::types::IndexId; -use quickwit_proto::{metastore, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; +use quickwit_proto::{metastore, IndexId, IndexUid, NodeId, NodeIdRef, ShardId, SourceId}; use serde::Serialize; use tracing::{error, info}; @@ -249,6 +248,7 @@ impl ControlPlaneModel { } /// Removes the shards identified by their index UID, source ID, and shard IDs. + #[allow(dead_code)] // Will remove this in a future PR. pub fn delete_shards( &mut self, index_uid: &IndexUid, @@ -261,6 +261,7 @@ impl ControlPlaneModel { /// Sets the state of the shards identified by their index UID, source ID, and shard IDs to /// `Closed`. + #[allow(dead_code)] // Will remove this in a future PR. pub fn close_shards( &mut self, index_uid: &IndexUid, @@ -425,6 +426,7 @@ impl ShardTable { /// Sets the state of the shards identified by their index UID, source ID, and shard IDs to /// `Closed`. + #[allow(dead_code)] // Will remove this in a future PR. pub fn close_shards( &mut self, index_uid: &IndexUid, @@ -517,7 +519,7 @@ mod tests { source_id: source_id.clone(), shard_id: 2, leader_id: "test-leader-0".to_string(), - shard_state: ShardState::Closing as i32, + shard_state: ShardState::Unavailable as i32, ..Default::default() }; let shard_03 = Shard { diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 7ec8524dfd8..5cb0f9a2801 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -33,8 +33,7 @@ use quickwit_proto::ingest::IngestV2Error; use quickwit_proto::metastore::{ EntityKind, MetastoreError, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::types::NodeId; -use quickwit_proto::{metastore, IndexUid}; +use quickwit_proto::{metastore, IndexUid, NodeId}; use rand::seq::SliceRandom; use tokio::time::timeout; diff --git a/quickwit/quickwit-control-plane/src/lib.rs b/quickwit/quickwit-control-plane/src/lib.rs index 738615fe3ac..2e34be559f1 100644 --- a/quickwit/quickwit-control-plane/src/lib.rs +++ b/quickwit/quickwit-control-plane/src/lib.rs @@ -23,14 +23,9 @@ pub mod indexing_plan; pub mod indexing_scheduler; pub mod ingest; -use async_trait::async_trait; -use quickwit_common::pubsub::EventSubscriber; use quickwit_common::tower::Pool; -use quickwit_proto::control_plane::{ControlPlaneService, ControlPlaneServiceClient}; use quickwit_proto::indexing::{IndexingServiceClient, IndexingTask}; -use quickwit_proto::metastore::{CloseShardsRequest, DeleteShardsRequest}; -use quickwit_proto::{IndexUid, SourceId}; -use tracing::error; +use quickwit_proto::types::{IndexUid, SourceId}; /// It can however appear only once in a given index. /// In itself, `SourceId` is not unique, but the pair `(IndexUid, SourceId)` is. @@ -49,40 +44,5 @@ pub struct IndexerNodeInfo { pub type IndexerPool = Pool; -/// Subscribes to various metastore events and forwards them to the control plane using the inner -/// client. The actual subscriptions are set up in `quickwit-serve`. -#[derive(Debug, Clone)] -pub struct ControlPlaneEventSubscriber(ControlPlaneServiceClient); - -impl ControlPlaneEventSubscriber { - pub fn new(control_plane: ControlPlaneServiceClient) -> Self { - Self(control_plane) - } -} - -#[async_trait] -impl EventSubscriber for ControlPlaneEventSubscriber { - async fn handle_event(&mut self, request: CloseShardsRequest) { - if let Err(error) = self.0.close_shards(request).await { - error!( - "failed to notify control plane of close shards event: `{}`", - error - ); - } - } -} - -#[async_trait] -impl EventSubscriber for ControlPlaneEventSubscriber { - async fn handle_event(&mut self, request: DeleteShardsRequest) { - if let Err(error) = self.0.delete_shards(request).await { - error!( - "failed to notify control plane of delete shards event: `{}`", - error - ); - } - } -} - #[cfg(test)] mod tests; diff --git a/quickwit/quickwit-indexing/src/actors/publisher.rs b/quickwit/quickwit-indexing/src/actors/publisher.rs index d0d2586ade8..12c9c7b4205 100644 --- a/quickwit/quickwit-indexing/src/actors/publisher.rs +++ b/quickwit/quickwit-indexing/src/actors/publisher.rs @@ -193,11 +193,11 @@ impl Handler for Publisher { mod tests { use quickwit_actors::Universe; use quickwit_metastore::checkpoint::{ - IndexCheckpointDelta, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, + IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, }; use quickwit_metastore::{PublishSplitsRequestExt, SplitMetadata}; use quickwit_proto::metastore::EmptyResponse; - use quickwit_proto::IndexUid; + use quickwit_proto::{IndexUid, Position}; use tracing::Span; use super::*; diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index be116722b0c..ecd78847d94 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -34,7 +34,7 @@ use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_metastore::{SplitMetadata, StageSplitsRequestExt}; use quickwit_proto::metastore::{MetastoreService, MetastoreServiceClient, StageSplitsRequest}; use quickwit_proto::search::{ReportSplit, ReportSplitsRequest}; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::types::{IndexUid, PublishToken}; use quickwit_storage::SplitPayloadBuilder; use serde::Serialize; use tantivy::TrackedObject; diff --git a/quickwit/quickwit-indexing/src/models/indexed_split.rs b/quickwit/quickwit-indexing/src/models/indexed_split.rs index 45f2abc133b..d5182800f61 100644 --- a/quickwit/quickwit-indexing/src/models/indexed_split.rs +++ b/quickwit/quickwit-indexing/src/models/indexed_split.rs @@ -24,7 +24,7 @@ use quickwit_common::io::IoControls; use quickwit_common::temp_dir::TempDirectory; use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::types::{IndexUid, PublishToken}; use tantivy::directory::MmapDirectory; use tantivy::{IndexBuilder, TrackedObject}; use tracing::{instrument, Span}; diff --git a/quickwit/quickwit-indexing/src/models/packaged_split.rs b/quickwit/quickwit-indexing/src/models/packaged_split.rs index 1c0483fd2e8..8f7225f89c4 100644 --- a/quickwit/quickwit-indexing/src/models/packaged_split.rs +++ b/quickwit/quickwit-indexing/src/models/packaged_split.rs @@ -23,7 +23,7 @@ use std::fmt; use itertools::Itertools; use quickwit_common::temp_dir::TempDirectory; use quickwit_metastore::checkpoint::IndexCheckpointDelta; -use quickwit_proto::{IndexUid, PublishToken, SplitId}; +use quickwit_proto::types::{IndexUid, PublishToken, SplitId}; use tantivy::TrackedObject; use tracing::Span; diff --git a/quickwit/quickwit-indexing/src/models/publisher_message.rs b/quickwit/quickwit-indexing/src/models/publisher_message.rs index e6c58b82282..0642e3f80dc 100644 --- a/quickwit/quickwit-indexing/src/models/publisher_message.rs +++ b/quickwit/quickwit-indexing/src/models/publisher_message.rs @@ -22,7 +22,7 @@ use std::fmt; use itertools::Itertools; use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_metastore::SplitMetadata; -use quickwit_proto::{IndexUid, PublishToken}; +use quickwit_proto::types::{IndexUid, PublishToken}; use tantivy::TrackedObject; use tracing::Span; diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index 4d014adc8eb..9ca23c0f7b3 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -29,7 +29,8 @@ use bytes::Bytes; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_common::uri::Uri; use quickwit_config::FileSourceParams; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; +use quickwit_proto::types::Position; use serde::Serialize; use tokio::io::{AsyncBufReadExt, AsyncRead, BufReader}; use tracing::info; @@ -140,11 +141,14 @@ impl TypedSourceFactory for FileSourceFactory { let mut offset = 0; let reader: Box = if let Some(filepath) = ¶ms.filepath { let partition_id = PartitionId::from(filepath.to_string_lossy().to_string()); - if let Some(Position::Offset(offset_str)) = - checkpoint.position_for_partition(&partition_id).cloned() - { - offset = offset_str.parse::()?; - } + offset = checkpoint + .position_for_partition(&partition_id) + .map(|position| { + position + .as_usize() + .expect("file offset should be stored as usize") + }) + .unwrap_or(0); let (dir_uri, file_name) = dir_and_filename(filepath)?; let storage = ctx.storage_resolver.resolve(&dir_uri).await?; let file_size = storage.file_num_bytes(file_name).await?.try_into().unwrap(); diff --git a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs index 8b80d6b7726..5402ab69049 100644 --- a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs +++ b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs @@ -31,7 +31,8 @@ use google_cloud_pubsub::subscription::Subscription; use quickwit_actors::{ActorContext, ActorExitStatus, Mailbox}; use quickwit_common::rand::append_random_suffix; use quickwit_config::GcpPubSubSourceParams; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; +use quickwit_proto::types::Position; use serde_json::{json, Value as JsonValue}; use tokio::time; use tracing::{debug, info, warn}; diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index a068b7b0c11..2ed6c389be5 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -27,15 +27,14 @@ use async_trait::async_trait; use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_ingest::{decoded_mrecords, IngesterPool, MRecord, MultiFetchStream}; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::ingest::ingester::{ FetchResponseV2, IngesterService, TruncateRequest, TruncateSubrequest, }; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsSubrequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::types::NodeId; -use quickwit_proto::{IndexUid, PublishToken, ShardId, SourceId}; +use quickwit_proto::{IndexUid, NodeId, Position, PublishToken, ShardId, SourceId}; use serde_json::json; use tokio::time; use tracing::{debug, error, info, warn}; @@ -162,7 +161,10 @@ impl IngestSource { .assigned_shards .get_mut(&fetch_response.shard_id) .expect("shard should be assigned"); + 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(); for mrecord in decoded_mrecords(mrecord_batch) { match mrecord { @@ -172,16 +174,14 @@ impl IngestSource { MRecord::Commit => { batch_builder.force_commit(); } + MRecord::Eof => { + break; + } MRecord::Unknown => { bail!("source cannot decode mrecord"); } } } - let from_position_exclusive = assigned_shard.current_position_inclusive.clone(); - let to_position_inclusive = fetch_response - .to_position_inclusive() - .map(Position::from) - .unwrap_or(Position::Beginning); batch_builder .checkpoint_delta .record_partition_delta( @@ -198,8 +198,8 @@ impl IngestSource { let mut per_ingester_truncate_subrequests: HashMap<&NodeId, Vec> = HashMap::new(); - for (shard_id, truncate_position) in truncation_point { - if matches!(truncate_position, Position::Beginning) { + for (shard_id, to_position_exclusive) in truncation_point { + if matches!(to_position_exclusive, Position::Beginning) { continue; } let Some(shard) = self.assigned_shards.get(shard_id) else { @@ -209,15 +209,11 @@ impl IngestSource { ); continue; }; - let to_position_inclusive = truncate_position - .as_u64() - .expect("position should be a u64"); - let truncate_subrequest = TruncateSubrequest { index_uid: self.client_id.index_uid.clone().into(), source_id: self.client_id.source_id.clone(), shard_id: *shard_id, - to_position_inclusive, + to_position_inclusive: Some(to_position_exclusive.clone()), }; if let Some(follower_id) = &shard.follower_id_opt { per_ingester_truncate_subrequests @@ -371,10 +367,11 @@ impl Source for IngestSource { let source_id: SourceId = acquired_shard.source_id; let shard_id = acquired_shard.shard_id; let partition_id = PartitionId::from(shard_id); - let current_position_inclusive = - Position::from(acquired_shard.publish_position_inclusive); - let from_position_exclusive = current_position_inclusive.as_u64(); - let to_position_inclusive = None; + let current_position_inclusive = acquired_shard + .publish_position_inclusive + .unwrap_or_default(); + let from_position_exclusive = current_position_inclusive.clone(); + let to_position_inclusive = Position::Eof; if let Err(error) = ctx .protect_future(self.fetch_stream.subscribe( @@ -471,32 +468,21 @@ mod tests { let subrequest = &request.subrequests[0]; assert_eq!(subrequest.index_uid, "test-index:0"); assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, vec![1, 2, 3]); + 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, - publish_position_inclusive: "00000000000000000011".to_string(), - ..Default::default() - }, - Shard { - leader_id: "test-ingester-1".to_string(), - follower_id: None, - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 2, - publish_position_inclusive: "00000000000000000022".to_string(), - ..Default::default() - }, - ], + 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, + publish_position_inclusive: Some(11u64.into()), + ..Default::default() + }], }], }; Ok(response) @@ -515,7 +501,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(11)); + assert_eq!(request.from_position_exclusive, Some(11u64.into())); let (_service_stream_tx, service_stream) = ServiceStream::new_bounded(1); Ok(service_stream) @@ -531,7 +517,7 @@ 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, 11); + assert_eq!(subrequest.to_position_inclusive, Some(11u64.into())); let response = TruncateResponse {}; Ok(response) @@ -561,10 +547,9 @@ 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 and 2 and will fail to - // subscribe to shard 2. + // In this scenario, the indexer will only be able to acquire shard 1. let assignment = Assignment { - shard_ids: vec![1, 2, 3], + shard_ids: vec![1, 2], }; let publish_lock = source.publish_lock.clone(); let publish_token = source.publish_token.clone(); @@ -598,7 +583,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: Position::from(11u64), + current_position_inclusive: 11u64.into(), }; assert_eq!(assigned_shard, &expected_assigned_shard); @@ -646,7 +631,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: Position::from(11u64), + current_position_inclusive: 11u64.into(), }, ); source.assigned_shards.insert( @@ -655,7 +640,7 @@ mod tests { leader_id: "test-ingester-1".into(), follower_id_opt: None, partition_id: 2u64.into(), - current_position_inclusive: Position::from(22u64), + current_position_inclusive: 22u64.into(), }, ); let fetch_response_tx = source.fetch_stream.fetch_response_tx(); @@ -665,11 +650,12 @@ mod tests { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 1, - from_position_inclusive: 12, 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(); @@ -679,11 +665,12 @@ mod tests { index_uid: "test-index:0".into(), source_id: "test-source".into(), shard_id: 2, - from_position_inclusive: 23, mrecord_batch: Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc-223"), - mrecord_lengths: vec![14], + mrecord_buffer: Bytes::from_static(b"\0\0test-doc-223\0\x01"), + mrecord_lengths: vec![14, 2], }), + from_position_exclusive: Some(22u64.into()), + to_position_inclusive: Some(Position::Eof), })) .await .unwrap(); @@ -710,12 +697,12 @@ mod tests { assert_eq!(partition_deltas.len(), 2); assert_eq!(partition_deltas[0].0, 1u64.into()); - assert_eq!(partition_deltas[0].1.from, 11u64.into()); - assert_eq!(partition_deltas[0].1.to, 14u64.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[1].0, 2u64.into()); - assert_eq!(partition_deltas[1].1.from, 22u64.into()); - assert_eq!(partition_deltas[1].1.to, 23u64.into()); + assert_eq!(partition_deltas[1].1.from, Position::from(22u64)); + assert_eq!(partition_deltas[1].1.to, Position::Eof); } #[tokio::test] @@ -741,15 +728,15 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 1); - assert_eq!(subrequest_0.to_position_inclusive, 11); + assert_eq!(subrequest_0.to_position_inclusive, Some(11u64.into())); let subrequest_1 = &request.subrequests[1]; assert_eq!(subrequest_1.shard_id, 2); - assert_eq!(subrequest_1.to_position_inclusive, 22); + assert_eq!(subrequest_1.to_position_inclusive, Some(22u64.into())); let subrequest_2 = &request.subrequests[2]; assert_eq!(subrequest_2.shard_id, 3); - assert_eq!(subrequest_2.to_position_inclusive, 33); + assert_eq!(subrequest_2.to_position_inclusive, Some(33u64.into())); Ok(TruncateResponse {}) }); @@ -766,11 +753,11 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 2); - assert_eq!(subrequest_0.to_position_inclusive, 22); + assert_eq!(subrequest_0.to_position_inclusive, Some(22u64.into())); let subrequest_1 = &request.subrequests[1]; assert_eq!(subrequest_1.shard_id, 3); - assert_eq!(subrequest_1.to_position_inclusive, 33); + assert_eq!(subrequest_1.to_position_inclusive, Some(33u64.into())); Ok(TruncateResponse {}) }); @@ -787,7 +774,7 @@ mod tests { let subrequest_0 = &request.subrequests[0]; assert_eq!(subrequest_0.shard_id, 4); - assert_eq!(subrequest_0.to_position_inclusive, 44); + assert_eq!(subrequest_0.to_position_inclusive, Some(44u64.into())); Ok(TruncateResponse {}) }); @@ -820,7 +807,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: None, partition_id: 1u64.into(), - current_position_inclusive: Position::from(11u64), + current_position_inclusive: 11u64.into(), }, ); source.assigned_shards.insert( @@ -829,7 +816,7 @@ mod tests { leader_id: "test-ingester-0".into(), follower_id_opt: Some("test-ingester-1".into()), partition_id: 2u64.into(), - current_position_inclusive: Position::from(22u64), + current_position_inclusive: 22u64.into(), }, ); source.assigned_shards.insert( @@ -838,7 +825,7 @@ mod tests { leader_id: "test-ingester-1".into(), follower_id_opt: Some("test-ingester-0".into()), partition_id: 3u64.into(), - current_position_inclusive: Position::from(33u64), + current_position_inclusive: 33u64.into(), }, ); source.assigned_shards.insert( @@ -847,7 +834,7 @@ mod tests { leader_id: "test-ingester-2".into(), follower_id_opt: Some("test-ingester-3".into()), partition_id: 4u64.into(), - current_position_inclusive: Position::from(44u64), + current_position_inclusive: 44u64.into(), }, ); source.assigned_shards.insert( diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index 06798f7c62e..49f7bbff10e 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -27,7 +27,8 @@ use quickwit_ingest::{ get_ingest_api_service, CreateQueueIfNotExistsRequest, DocCommand, FetchRequest, FetchResponse, GetPartitionId, IngestApiService, SuggestTruncateRequest, }; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; +use quickwit_proto::types::Position; use serde::Serialize; use serde_json::Value as JsonValue; @@ -84,13 +85,9 @@ impl IngestApiSource { }; ingest_api_service.ask_for_res(create_queue_req).await?; - let previous_offset = if let Some(Position::Offset(offset_str)) = - checkpoint.position_for_partition(&partition_id) - { - Some(offset_str.parse::()?) - } else { - None - }; + let previous_offset = checkpoint + .position_for_partition(&partition_id) + .map(|position| position.as_u64().expect("offset should be stored as u64")); let current_offset = previous_offset; let ingest_api_source = IngestApiSource { ctx, @@ -175,10 +172,10 @@ impl Source for IngestApiSource { checkpoint: SourceCheckpoint, ctx: &ActorContext, ) -> anyhow::Result<()> { - if let Some(Position::Offset(offset_str)) = + if let Some(Position::Offset(offset)) = checkpoint.position_for_partition(&self.partition_id) { - let up_to_position_included = offset_str.parse::()?; + let up_to_position_included = offset.as_u64().expect("offset should be stored as u64"); let suggest_truncate_req = SuggestTruncateRequest { index_id: self.ctx.index_id().to_string(), up_to_position_included, diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index 0b30531f009..0fde5b4fc98 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -29,10 +29,10 @@ use itertools::Itertools; use oneshot; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_config::KafkaSourceParams; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService}; -use quickwit_proto::IndexUid; +use quickwit_proto::{IndexUid, Position}; use rdkafka::config::{ClientConfig, RDKafkaLogLevel}; use rdkafka::consumer::{ BaseConsumer, CommitMode, Consumer, ConsumerContext, DefaultConsumerContext, Rebalance, @@ -349,7 +349,7 @@ impl KafkaSource { assignment_tx: oneshot::Sender>, ) -> anyhow::Result<()> { let index_metadata_request = - IndexMetadataRequest::for_index_uid(self.ctx.index_uid().to_string()); + IndexMetadataRequest::for_index_uid(self.ctx.index_uid().clone()); let index_metadata = ctx .protect_future( self.ctx @@ -379,12 +379,15 @@ impl KafkaSource { .unwrap_or(Position::Beginning); let next_offset = match ¤t_position { Position::Beginning => Offset::Beginning, - Position::Offset(_) => { - let offset = current_position + Position::Offset(offset) => { + let offset = offset .as_i64() - .expect("Kafka offset should be stored as i64."); + .expect("Kafka offset should be stored as i64"); Offset::Offset(offset + 1) } + Position::Eof => { + panic!("position of a Kafka partition should never be EOF") + } }; self.state .assigned_partitions diff --git a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs index 807e692194f..7b3ba961781 100644 --- a/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs +++ b/quickwit/quickwit-indexing/src/source/kinesis/kinesis_source.rs @@ -31,9 +31,8 @@ use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_aws::get_aws_config; use quickwit_common::retry::RetryParams; use quickwit_config::{KinesisSourceParams, RegionOrEndpoint}; -use quickwit_metastore::checkpoint::{ - PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, -}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint, SourceCheckpointDelta}; +use quickwit_proto::types::Position; use serde_json::{json, Value as JsonValue}; use tokio::sync::mpsc; use tokio::time; @@ -144,15 +143,16 @@ impl KinesisSource { fn spawn_shard_consumer(&mut self, ctx: &SourceContext, shard_id: ShardId) { assert!(!self.state.shard_consumers.contains_key(&shard_id)); - let partition_id = PartitionId::from(shard_id.as_ref()); + let partition_id = PartitionId::from(shard_id.as_str()); let position = self .checkpoint .position_for_partition(&partition_id) .cloned() .unwrap_or(Position::Beginning); let from_sequence_number_exclusive = match &position { - Position::Offset(offset) => Some(offset.to_string()), Position::Beginning => None, + Position::Offset(offset) => Some(offset.to_string()), + Position::Eof => panic!("position of a Kinesis shard should never be EOF"), }; let shard_consumer = ShardConsumer::new( self.stream_name.clone(), diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index 9f418eaeaa4..50b91b12928 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -33,8 +33,8 @@ use pulsar::{ }; use quickwit_actors::{ActorContext, ActorExitStatus, Mailbox}; use quickwit_config::{PulsarSourceAuth, PulsarSourceParams}; -use quickwit_metastore::checkpoint::{PartitionId, Position, SourceCheckpoint}; -use quickwit_proto::IndexUid; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; +use quickwit_proto::types::{IndexUid, Position}; use serde_json::{json, Value as JsonValue}; use tokio::time; use tracing::{debug, info, warn}; @@ -439,7 +439,7 @@ mod pulsar_broker_tests { use quickwit_common::rand::append_random_suffix; use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; use quickwit_metastore::checkpoint::{ - IndexCheckpointDelta, PartitionId, Position, SourceCheckpointDelta, + IndexCheckpointDelta, PartitionId, SourceCheckpointDelta, }; use quickwit_metastore::{ metastore_for_test, CreateIndexRequestExt, SplitMetadata, StageSplitsRequestExt, diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index 4f233757ab5..ceb42d0547a 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -24,9 +24,8 @@ use std::time::Duration; use async_trait::async_trait; use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_config::VecSourceParams; -use quickwit_metastore::checkpoint::{ - PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, -}; +use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint, SourceCheckpointDelta}; +use quickwit_proto::types::Position; use serde_json::Value as JsonValue; use tracing::info; @@ -43,7 +42,9 @@ pub struct VecSource { impl fmt::Debug for VecSource { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "VecSource {{ source_id: {} }}", self.source_id) + f.debug_struct("VecSource") + .field("source_id", &self.source_id) + .finish() } } @@ -59,10 +60,15 @@ impl TypedSourceFactory for VecSourceFactory { checkpoint: SourceCheckpoint, ) -> anyhow::Result { let partition = PartitionId::from(params.partition.as_str()); - let next_item_idx = match checkpoint.position_for_partition(&partition) { - Some(Position::Offset(offset_str)) => offset_str.parse::()? + 1, - Some(Position::Beginning) | None => 0, - }; + let next_item_idx = checkpoint + .position_for_partition(&partition) + .map(|position| { + position + .as_usize() + .expect("offset should be stored as usize") + + 1 + }) + .unwrap_or(0); Ok(VecSource { source_id: ctx.source_id().to_string(), next_item_idx, @@ -76,7 +82,7 @@ fn position_from_offset(offset: usize) -> Position { if offset == 0 { return Position::Beginning; } - Position::from(offset as u64 - 1) + Position::from(offset - 1) } #[async_trait] @@ -95,7 +101,7 @@ impl Source for VecSource { .cloned(), ); if doc_batch.docs.is_empty() { - info!("Reached end of source."); + info!("reached end of source"); ctx.send_exit_with_success(batch_sink).await?; return Err(ActorExitStatus::Success); } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index 2c7b641247d..40d0d3ae37e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -21,7 +21,7 @@ use std::borrow::Borrow; use std::collections::hash_map::Entry; use std::collections::HashMap; use std::fmt; -use std::ops::RangeInclusive; +use std::ops::{Bound, RangeBounds}; use std::sync::Arc; use bytes::{BufMut, BytesMut}; @@ -29,14 +29,13 @@ use futures::StreamExt; use quickwit_common::ServiceStream; use quickwit_proto::ingest::ingester::{FetchResponseV2, IngesterService, OpenFetchStreamRequest}; use quickwit_proto::ingest::{IngestV2Error, IngestV2Result, MRecordBatch}; -use quickwit_proto::types::{queue_id, NodeId, QueueId, ShardId, SourceId}; -use quickwit_proto::IndexUid; +use quickwit_proto::types::{queue_id, IndexUid, NodeId, Position, QueueId, ShardId, SourceId}; use tokio::sync::{mpsc, watch, RwLock}; use tokio::task::JoinHandle; use tracing::{debug, error, warn}; use super::ingester::IngesterState; -use super::models::ShardStatus; +use crate::ingest_v2::mrecord::is_eof_mrecord; use crate::{ClientId, IngesterPool}; /// A fetch task is responsible for waiting and pushing new records written to a shard's record log @@ -48,14 +47,13 @@ pub(super) struct FetchTask { source_id: SourceId, shard_id: ShardId, queue_id: QueueId, - /// Range of records to fetch. When there is no upper bound, the end of the range is set to - /// `u64::MAX`. - fetch_range: RangeInclusive, + /// Range of records to fetch. + fetch_range: FetchRange, state: Arc>, fetch_response_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 log unnecessarily. - shard_status_rx: watch::Receiver, + /// task does not need to grab the lock and poll the mrecordlog queue unnecessarily. + new_records_rx: watch::Receiver<()>, batch_num_bytes: usize, } @@ -70,12 +68,7 @@ impl fmt::Debug for FetchTask { } } -// TODO: Drop when `Iterator::advance_by()` is stabilized. -fn advance_by(range: &mut RangeInclusive, len: u64) { - *range = *range.start() + len..=*range.end(); -} - -type FetchTaskHandle = JoinHandle<(u64, Option)>; +type FetchTaskHandle = JoinHandle<(u64, Position)>; impl FetchTask { pub const DEFAULT_BATCH_NUM_BYTES: usize = 1024 * 1024; // 1 MiB @@ -83,30 +76,27 @@ impl FetchTask { pub fn spawn( open_fetch_stream_request: OpenFetchStreamRequest, state: Arc>, - shard_status_rx: watch::Receiver, + new_records_rx: watch::Receiver<()>, batch_num_bytes: usize, ) -> ( ServiceStream>, FetchTaskHandle, ) { let (fetch_response_tx, fetch_stream) = ServiceStream::new_bounded(3); - let from_position_inclusive = open_fetch_stream_request - .from_position_exclusive - .map(|position| position + 1) - .unwrap_or(0); - let to_position_inclusive = open_fetch_stream_request - .to_position_inclusive - .unwrap_or(u64::MAX); + let fetch_range = FetchRange::new( + open_fetch_stream_request.from_position_exclusive(), + open_fetch_stream_request.to_position_inclusive(), + ); let mut fetch_task = Self { queue_id: open_fetch_stream_request.queue_id(), client_id: open_fetch_stream_request.client_id, index_uid: open_fetch_stream_request.index_uid.into(), source_id: open_fetch_stream_request.source_id, shard_id: open_fetch_stream_request.shard_id, - fetch_range: from_position_inclusive..=to_position_inclusive, + fetch_range, state, fetch_response_tx, - shard_status_rx, + new_records_rx, batch_num_bytes, }; let future = async move { fetch_task.run().await }; @@ -114,33 +104,11 @@ impl FetchTask { (fetch_stream, fetch_task_handle) } - /// Waits for new records. Returns `false` if the ingester is dropped. - async fn wait_for_new_records(&mut self) -> bool { - loop { - let shard_status = self.shard_status_rx.borrow().clone(); - - if shard_status.shard_state.is_closed() - && shard_status.publish_position_inclusive <= *self.fetch_range.start() - { - // The shard is closed and we have fetched all records up to the publish position. - return false; - } - if shard_status.replication_position_inclusive >= *self.fetch_range.start() { - // Some new records are available. - return true; - } - if self.shard_status_rx.changed().await.is_err() { - // The ingester was dropped. - return false; - } - } - } - /// Runs the fetch task. It waits for new records in the log and pushes them into the fetch /// response channel until `to_position_inclusive` is reached, the shard is closed and /// `to_position_inclusive` is reached, or the ingester is dropped. It returns the total number /// of records fetched and the position of the last record fetched. - async fn run(&mut self) -> (u64, Option) { + async fn run(&mut self) -> (u64, Position) { debug!( client_id=%self.client_id, index_uid=%self.index_uid, @@ -149,16 +117,25 @@ impl FetchTask { fetch_range=?self.fetch_range, "spawning fetch task" ); - let mut total_num_records = 0; + let mut has_drained_queue = true; + let mut has_reached_eof = false; + let mut num_records_total = 0; - while !self.fetch_range.is_empty() { - if !self.wait_for_new_records().await { + while !has_reached_eof && !self.fetch_range.is_empty() { + if has_drained_queue && self.new_records_rx.changed().await.is_err() { + // The shard was dropped. break; } - let fetch_range = self.fetch_range.clone(); + has_drained_queue = true; + let mut mrecord_buffer = BytesMut::with_capacity(self.batch_num_bytes); + let mut mrecord_lengths = Vec::new(); + let state_guard = self.state.read().await; - let Ok(mrecords) = state_guard.mrecordlog.range(&self.queue_id, fetch_range) else { + let Ok(mrecords) = state_guard + .mrecordlog + .range(&self.queue_id, self.fetch_range) + else { warn!( client_id=%self.client_id, index_uid=%self.index_uid, @@ -168,11 +145,9 @@ impl FetchTask { ); break; }; - let mut mrecord_buffer = BytesMut::with_capacity(self.batch_num_bytes); - let mut mrecord_lengths = Vec::new(); - for (_position, mrecord) in mrecords { if mrecord_buffer.len() + mrecord.len() > mrecord_buffer.capacity() { + has_drained_queue = false; break; } mrecord_buffer.put(mrecord.borrow()); @@ -181,29 +156,47 @@ impl FetchTask { // Drop the lock while we send the message. drop(state_guard); + if mrecord_buffer.is_empty() { + continue; + } + 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..]; + + has_reached_eof = is_eof_mrecord(last_mrecord); + let mrecord_batch = MRecordBatch { mrecord_buffer: mrecord_buffer.freeze(), mrecord_lengths, }; let num_records = mrecord_batch.num_mrecords() as u64; - total_num_records += num_records; + num_records_total += num_records; + + let from_position_exclusive = self.fetch_range.from_position_exclusive(); + self.fetch_range.advance_by(num_records); + let to_position_inclusive = if has_reached_eof { + Position::Eof + } else { + self.fetch_range.from_position_exclusive() + }; 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_inclusive: *self.fetch_range.start(), + from_position_exclusive: Some(from_position_exclusive), + to_position_inclusive: Some(to_position_inclusive), }; - advance_by(&mut self.fetch_range, num_records); - if self .fetch_response_tx .send(Ok(fetch_response)) .await .is_err() { - // The ingester was dropped. + // The consumer was dropped. break; } } @@ -214,11 +207,10 @@ impl FetchTask { shard_id=%self.shard_id, "fetch task completed" ); - if total_num_records == 0 { - (0, None) - } else { - (total_num_records, Some(*self.fetch_range.start() - 1)) - } + ( + num_records_total, + self.fetch_range.from_position_exclusive(), + ) } } @@ -260,8 +252,8 @@ impl MultiFetchStream { index_uid: IndexUid, source_id: SourceId, shard_id: ShardId, - from_position_exclusive: Option, - to_position_inclusive: Option, + from_position_exclusive: Position, + to_position_inclusive: Position, ) -> IngestV2Result<()> { let queue_id = queue_id(index_uid.as_str(), &source_id, shard_id); let entry = self.fetch_task_handles.entry(queue_id.clone()); @@ -271,81 +263,25 @@ impl MultiFetchStream { "stream has already subscribed to shard `{queue_id}`" ))); } - let (mut preferred_ingester_id, mut failover_ingester_id) = + let (preferred_ingester_id, failover_ingester_id_opt) = select_preferred_and_failover_ingesters(&self.self_node_id, leader_id, follower_id_opt); - // Obtain a fetch stream from the preferred or failover ingester. - let fetch_stream = loop { - let Some(mut ingester) = self.ingester_pool.get(&preferred_ingester_id) else { - if let Some(failover_ingester_id) = failover_ingester_id.take() { - warn!( - client_id=%self.client_id, - index_uid=%index_uid, - source_id=%source_id, - shard_id=%shard_id, - "Ingester `{preferred_ingester_id}` is not available. Failing over to ingester `{failover_ingester_id}`." - ); - preferred_ingester_id = failover_ingester_id; - continue; - }; - return Err(IngestV2Error::Internal(format!( - "shard `{queue_id}` is unavailable" - ))); - }; - let open_fetch_stream_request = OpenFetchStreamRequest { - client_id: self.client_id.clone(), - index_uid: index_uid.clone().into(), - source_id: source_id.clone(), - shard_id, - from_position_exclusive, - to_position_inclusive, - }; - match ingester.open_fetch_stream(open_fetch_stream_request).await { - Ok(fetch_stream) => { - break fetch_stream; - } - Err(error) => { - if let Some(failover_ingester_id) = failover_ingester_id.take() { - warn!( - client_id=%self.client_id, - index_uid=%index_uid, - source_id=%source_id, - shard_id=%shard_id, - error=?error, - "Failed to open fetch stream from `{preferred_ingester_id}`. Failing over to ingester `{failover_ingester_id}`." - ); - preferred_ingester_id = failover_ingester_id; - continue; - }; - error!( - client_id=%self.client_id, - index_uid=%index_uid, - source_id=%source_id, - shard_id=%shard_id, - error=?error, - "Failed to open fetch stream from `{preferred_ingester_id}`." - ); - return Err(IngestV2Error::Internal(format!( - "shard `{queue_id}` is unavailable" - ))); - } - }; - }; - let client_id = self.client_id.clone(); - let ingester_pool = self.ingester_pool.clone(); - let fetch_response_tx = self.fetch_response_tx.clone(); + let mut ingester_ids = Vec::with_capacity(1 + failover_ingester_id_opt.is_some() as usize); + ingester_ids.push(preferred_ingester_id); + + if let Some(failover_ingester_id) = failover_ingester_id_opt { + ingester_ids.push(failover_ingester_id); + } let fetch_task_future = fault_tolerant_fetch_task( - client_id, + self.client_id.clone(), index_uid, source_id, shard_id, from_position_exclusive, to_position_inclusive, - preferred_ingester_id, - failover_ingester_id, - ingester_pool, - fetch_stream, - fetch_response_tx, + ingester_ids, + self.ingester_pool.clone(), + self.fetch_response_tx.clone(), ); let fetch_task_handle = tokio::spawn(fetch_task_future); self.fetch_task_handles.insert(queue_id, fetch_task_handle); @@ -423,79 +359,218 @@ async fn fault_tolerant_fetch_task( index_uid: IndexUid, source_id: SourceId, shard_id: ShardId, - mut from_position_exclusive: Option, - to_position_inclusive: Option, - - preferred_ingester_id: NodeId, - mut failover_ingester_id: Option, - + mut from_position_exclusive: Position, + to_position_inclusive: Position, + ingester_ids: Vec, ingester_pool: IngesterPool, - - mut fetch_stream: ServiceStream>, fetch_response_tx: mpsc::Sender>, ) { - while let Some(fetch_response_result) = fetch_stream.next().await { - match fetch_response_result { - Ok(fetch_response) => { - from_position_exclusive = fetch_response.to_position_inclusive(); - if fetch_response_tx.send(Ok(fetch_response)).await.is_err() { - // The stream was dropped. - break; + // TODO: We can probably simplify this code by breaking it into smaller functions. + 'outer: for (ingester_idx, ingester_id) in ingester_ids.iter().enumerate() { + let failover_ingester_id_opt = ingester_ids.get(ingester_idx + 1); + + let mut ingester = match ingester_pool.get(ingester_id) { + Some(ingester) => ingester, + _ => { + if let Some(failover_ingester_id) = failover_ingester_id_opt { + warn!( + client_id=%client_id, + index_uid=%index_uid, + source_id=%source_id, + shard_id=%shard_id, + "ingester `{ingester_id}` is not available: failing over to ingester `{failover_ingester_id}`" + ); + } else { + error!( + client_id=%client_id, + index_uid=%index_uid, + source_id=%source_id, + shard_id=%shard_id, + "ingester `{ingester_id}` is not available: closing fetch stream" + ); + let ingest_error = IngestV2Error::IngesterUnavailable { + ingester_id: ingester_id.clone(), + }; + // Attempt to send the error to the consumer in a best-effort manner before + // returning. + let _ = fetch_response_tx.send(Err(ingest_error)).await; } + continue; } + }; + let open_fetch_stream_request = OpenFetchStreamRequest { + client_id: client_id.clone(), + index_uid: index_uid.clone().into(), + source_id: source_id.clone(), + shard_id, + from_position_exclusive: Some(from_position_exclusive.clone()), + to_position_inclusive: Some(to_position_inclusive.clone()), + }; + let mut fetch_stream = match ingester.open_fetch_stream(open_fetch_stream_request).await { + Ok(fetch_stream) => fetch_stream, Err(ingest_error) => { - if let Some(failover_ingester_id) = failover_ingester_id.take() { + if let Some(failover_ingester_id) = failover_ingester_id_opt { warn!( client_id=%client_id, index_uid=%index_uid, source_id=%source_id, shard_id=%shard_id, - error=?ingest_error, - "Error fetching from `{preferred_ingester_id}`. Failing over to ingester `{failover_ingester_id}`." + error=%ingest_error, + "failed to open fetch stream from ingester `{ingester_id}`: failing over to ingester `{failover_ingester_id}`" ); - let mut ingester = ingester_pool - .get(&preferred_ingester_id) - .expect("TODO: handle error"); - let open_fetch_stream_request = OpenFetchStreamRequest { - client_id: client_id.clone(), - index_uid: index_uid.clone().into(), - source_id: source_id.clone(), - shard_id, - from_position_exclusive, - to_position_inclusive, - }; - fetch_stream = ingester - .open_fetch_stream(open_fetch_stream_request) - .await - .expect("TODO:"); - continue; + } else { + error!( + client_id=%client_id, + index_uid=%index_uid, + source_id=%source_id, + shard_id=%shard_id, + error=%ingest_error, + "failed to open fetch stream from ingester `{ingester_id}`: closing fetch stream" + ); + let _ = fetch_response_tx.send(Err(ingest_error)).await; + } + 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; + } + if to_position_inclusive == Position::Eof { + // The stream has reached the end of the shard. + return; + } + from_position_exclusive = to_position_inclusive; + } + Err(ingest_error) => { + if let Some(failover_ingester_id) = failover_ingester_id_opt { + warn!( + client_id=%client_id, + index_uid=%index_uid, + source_id=%source_id, + shard_id=%shard_id, + error=%ingest_error, + "failed to fetch records from ingester `{ingester_id}`: failing over to ingester `{failover_ingester_id}`" + ); + } else { + error!( + client_id=%client_id, + index_uid=%index_uid, + source_id=%source_id, + shard_id=%shard_id, + error=%ingest_error, + "failed to fetch records from ingester `{ingester_id}`: closing fetch stream" + ); + let _ = fetch_response_tx.send(Err(ingest_error)).await; + } + continue 'outer; } - error!( - client_id=%client_id, - index_uid=%index_uid, - source_id=%source_id, - shard_id=%shard_id, - error=?ingest_error, - "Error fetching from `{preferred_ingester_id}`." - ); - let _ = fetch_response_tx.send(Err(ingest_error)).await; - break; } } } } +#[derive(Debug, Clone, Copy)] +struct FetchRange { + from_position_exclusive_opt: Option, + to_position_inclusive_opt: Option, +} + +impl FetchRange { + fn new(from_position_exclusive: Position, to_position_inclusive: Position) -> Self { + Self { + from_position_exclusive_opt: from_position_exclusive.as_u64(), + to_position_inclusive_opt: to_position_inclusive.as_u64(), + } + } + + #[allow(clippy::wrong_self_convention)] + fn from_position_exclusive(&self) -> Position { + Position::from(self.from_position_exclusive_opt) + } + + fn is_empty(&self) -> bool { + match ( + self.from_position_exclusive_opt, + self.to_position_inclusive_opt, + ) { + (Some(from_position_exclusive), Some(to_position_inclusive)) => { + from_position_exclusive >= to_position_inclusive + } + _ => false, + } + } + + fn advance_by(&mut self, num_records: u64) { + if let Some(from_position_exclusive) = self.from_position_exclusive_opt { + self.from_position_exclusive_opt = Some(from_position_exclusive + num_records); + } else { + self.from_position_exclusive_opt = Some(num_records - 1); + } + } +} + +impl RangeBounds for FetchRange { + fn start_bound(&self) -> std::ops::Bound<&u64> { + self.from_position_exclusive_opt + .as_ref() + .map(Bound::Excluded) + .unwrap_or(Bound::Unbounded) + } + + fn end_bound(&self) -> std::ops::Bound<&u64> { + self.to_position_inclusive_opt + .as_ref() + .map(Bound::Included) + .unwrap_or(Bound::Unbounded) + } +} + #[cfg(test)] mod tests { use std::time::Duration; use bytes::Bytes; use mrecordlog::MultiRecordLog; - use quickwit_proto::ingest::ShardState; + use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::types::queue_id; use tokio::time::timeout; use super::*; + use crate::MRecord; + + #[test] + fn test_fetch_range() { + let mut fetch_range = FetchRange::new(Position::Beginning, Position::Eof); + assert_eq!(fetch_range.start_bound(), Bound::Unbounded); + assert_eq!(fetch_range.end_bound(), Bound::Unbounded); + assert_eq!(fetch_range.from_position_exclusive(), Position::Beginning); + assert!(!fetch_range.is_empty()); + + fetch_range.advance_by(1); + assert_eq!(fetch_range.start_bound(), Bound::Excluded(&0)); + assert_eq!(fetch_range.from_position_exclusive(), 0u64); + assert!(!fetch_range.is_empty()); + + fetch_range.advance_by(10); + assert_eq!(fetch_range.from_position_exclusive(), Position::from(10u64)); + + let mut fetch_range = FetchRange::new(Position::Beginning, Position::from(1u64)); + assert!(!fetch_range.is_empty()); + + fetch_range.advance_by(1); + assert_eq!(fetch_range.from_position_exclusive(), 0u64); + assert!(!fetch_range.is_empty()); + + fetch_range.advance_by(1); + assert_eq!(fetch_range.from_position_exclusive(), 1u64); + assert!(fetch_range.is_empty()); + } #[tokio::test] async fn test_fetch_task() { @@ -512,18 +587,17 @@ mod tests { from_position_exclusive: None, to_position_inclusive: None, }; - let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let (new_records_tx, new_records_rx) = watch::channel(()); let state = Arc::new(RwLock::new(IngesterState { mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), + shards: HashMap::new(), replication_streams: HashMap::new(), replication_tasks: HashMap::new(), })); let (mut fetch_stream, fetch_task_handle) = FetchTask::spawn( open_fetch_stream_request, state.clone(), - shard_status_rx, + new_records_rx, 1024, ); let queue_id = queue_id(&index_uid, &source_id, 1); @@ -537,17 +611,18 @@ mod tests { .unwrap(); state_guard .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) + .append_record(&queue_id, None, MRecord::new_doc("test-doc-foo").encode()) .await .unwrap(); - let shard_status = ShardStatus { - replication_position_inclusive: 0.into(), - ..Default::default() - }; - shard_status_tx.send(shard_status).unwrap(); drop(state_guard); - let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap_err(); + + new_records_tx.send(()).unwrap(); + + let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() @@ -555,14 +630,18 @@ mod tests { 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_inclusive, 0); + assert_eq!( + fetch_response.from_position_exclusive(), + Position::Beginning + ); + assert_eq!(fetch_response.to_position_inclusive(), 0u64); assert_eq!( fetch_response .mrecord_batch .as_ref() .unwrap() .mrecord_lengths, - [12] + [14] ); assert_eq!( fetch_response @@ -570,41 +649,44 @@ mod tests { .as_ref() .unwrap() .mrecord_buffer, - "test-doc-000" + "\0\0test-doc-foo" ); let mut state_guard = state.write().await; state_guard .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-001")) + .append_record(&queue_id, None, MRecord::new_doc("test-doc-bar").encode()) .await .unwrap(); drop(state_guard); - timeout(Duration::from_millis(100), fetch_stream.next()) + timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap_err(); - let shard_status = ShardStatus { - replication_position_inclusive: 1.into(), - ..Default::default() - }; - shard_status_tx.send(shard_status).unwrap(); + new_records_tx.send(()).unwrap(); + + // Trigger a spurious notification. + new_records_tx.send(()).unwrap(); - let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + let fetch_response = timeout(Duration::from_millis(50), fetch_stream.next()) .await .unwrap() .unwrap() .unwrap(); - assert_eq!(fetch_response.from_position_inclusive, 1); + assert_eq!( + fetch_response.from_position_exclusive(), + Position::from(0u64) + ); + assert_eq!(fetch_response.to_position_inclusive(), 1u64); assert_eq!( fetch_response .mrecord_batch .as_ref() .unwrap() .mrecord_lengths, - [12] + [14] ); assert_eq!( fetch_response @@ -612,19 +694,57 @@ mod tests { .as_ref() .unwrap() .mrecord_buffer, - "test-doc-001" + "\0\0test-doc-bar" ); - let shard_status = ShardStatus { - shard_state: ShardState::Closed, - replication_position_inclusive: 1.into(), - publish_position_inclusive: 1.into(), - }; - shard_status_tx.send(shard_status).unwrap(); + let mut state_guard = state.write().await; + + let mrecords = [ + MRecord::new_doc("test-doc-baz").encode(), + MRecord::new_doc("test-doc-qux").encode(), + MRecord::Eof.encode(), + ] + .into_iter(); + + state_guard + .mrecordlog + .append_records(&queue_id, None, mrecords) + .await + .unwrap(); + drop(state_guard); + + new_records_tx.send(()).unwrap(); + + let fetch_response = 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); + assert_eq!( + fetch_response + .mrecord_batch + .as_ref() + .unwrap() + .mrecord_lengths, + [14, 14, 2] + ); + assert_eq!( + fetch_response + .mrecord_batch + .as_ref() + .unwrap() + .mrecord_buffer, + "\0\0test-doc-baz\0\0test-doc-qux\0\x02" + ); let (num_records, last_position) = fetch_task_handle.await.unwrap(); - assert_eq!(num_records, 2); - assert_eq!(last_position, Some(1)); + assert_eq!(num_records, 5); + assert_eq!(last_position, Position::from(4u64)); } #[tokio::test] @@ -640,20 +760,19 @@ mod tests { source_id: source_id.clone(), shard_id: 1, from_position_exclusive: None, - to_position_inclusive: Some(0), + to_position_inclusive: Some(Position::from(0u64)), }; let state = Arc::new(RwLock::new(IngesterState { mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), + shards: HashMap::new(), replication_streams: HashMap::new(), replication_tasks: HashMap::new(), })); - let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let (new_records_tx, new_records_rx) = watch::channel(()); let (mut fetch_stream, fetch_task_handle) = FetchTask::spawn( open_fetch_stream_request, state.clone(), - shard_status_rx, + new_records_rx, 1024, ); let queue_id = queue_id(&index_uid, &source_id, 1); @@ -667,46 +786,31 @@ mod tests { .unwrap(); state_guard .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-foo")) .await .unwrap(); - - let shard_status = ShardStatus { - replication_position_inclusive: 0.into(), - ..Default::default() - }; - shard_status_tx.send(shard_status).unwrap(); drop(state_guard); - let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + new_records_tx.send(()).unwrap(); + + let fetch_response = 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); - assert_eq!(fetch_response.from_position_inclusive, 0); - assert_eq!( - fetch_response - .mrecord_batch - .as_ref() - .unwrap() - .mrecord_lengths, - [12] - ); assert_eq!( fetch_response .mrecord_batch .as_ref() .unwrap() - .mrecord_buffer, - "test-doc-000" + .mrecord_lengths + .len(), + 1 ); let (num_records, last_position) = fetch_task_handle.await.unwrap(); assert_eq!(num_records, 1); - assert_eq!(last_position, Some(0)); + assert_eq!(last_position, 0u64); } #[tokio::test] @@ -722,22 +826,17 @@ mod tests { source_id: source_id.clone(), shard_id: 1, from_position_exclusive: None, - to_position_inclusive: Some(2), + to_position_inclusive: Some(Position::from(2u64)), }; let state = Arc::new(RwLock::new(IngesterState { mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), + shards: HashMap::new(), replication_streams: HashMap::new(), replication_tasks: HashMap::new(), })); - let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); - let (mut fetch_stream, fetch_task_handle) = FetchTask::spawn( - open_fetch_stream_request, - state.clone(), - shard_status_rx, - 30, - ); + let (new_records_tx, new_records_rx) = watch::channel(()); + let (mut fetch_stream, _fetch_task_handle) = + FetchTask::spawn(open_fetch_stream_request, state.clone(), new_records_rx, 30); let queue_id = queue_id(&index_uid, &source_id, 1); let mut state_guard = state.write().await; @@ -747,38 +846,28 @@ mod tests { .create_queue(&queue_id) .await .unwrap(); + + let records = [ + Bytes::from_static(b"test-doc-foo"), + Bytes::from_static(b"test-doc-bar"), + Bytes::from_static(b"test-doc-baz"), + ] + .into_iter(); + state_guard .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) - .await - .unwrap(); - state_guard - .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-001")) - .await - .unwrap(); - state_guard - .mrecordlog - .append_record(&queue_id, None, Bytes::from_static(b"test-doc-002")) + .append_records(&queue_id, None, records) .await .unwrap(); - - let shard_status = ShardStatus { - replication_position_inclusive: 2.into(), - ..Default::default() - }; - shard_status_tx.send(shard_status).unwrap(); drop(state_guard); - let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + new_records_tx.send(()).unwrap(); + + let fetch_response = 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); - assert_eq!(fetch_response.from_position_inclusive, 0); assert_eq!( fetch_response .mrecord_batch @@ -793,17 +882,14 @@ mod tests { .as_ref() .unwrap() .mrecord_buffer, - "test-doc-000test-doc-001" + "test-doc-footest-doc-bar" ); - let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + + let fetch_response = 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); - assert_eq!(fetch_response.from_position_inclusive, 2); assert_eq!( fetch_response .mrecord_batch @@ -818,25 +904,155 @@ mod tests { .as_ref() .unwrap() .mrecord_buffer, - "test-doc-002" + "test-doc-baz" ); + } - let (num_records, last_position) = fetch_task_handle.await.unwrap(); - assert_eq!(num_records, 3); - assert_eq!(last_position, Some(2)); + #[test] + fn test_select_preferred_and_failover_ingesters() { + let self_node_id: NodeId = "test-ingester-0".into(); + + let (preferred, failover) = + select_preferred_and_failover_ingesters(&self_node_id, "test-ingester-0".into(), None); + assert_eq!(preferred, "test-ingester-0"); + assert!(failover.is_none()); + + let (preferred, failover) = select_preferred_and_failover_ingesters( + &self_node_id, + "test-ingester-0".into(), + Some("test-ingester-1".into()), + ); + assert_eq!(preferred, "test-ingester-0"); + assert_eq!(failover.unwrap(), "test-ingester-1"); + + let (preferred, failover) = select_preferred_and_failover_ingesters( + &self_node_id, + "test-ingester-1".into(), + Some("test-ingester-0".into()), + ); + assert_eq!(preferred, "test-ingester-0"); + assert_eq!(failover.unwrap(), "test-ingester-1"); } #[tokio::test] - async fn test_fault_tolerant_fetch_task() { - // TODO: Backport from original branch. + async fn test_fault_tolerant_fetch_task_happy_failover() { + let client_id = "test-client".to_string(); + let index_uid: IndexUid = "test-index:0".into(); + let source_id: SourceId = "test-source".into(); + let shard_id: ShardId = 1; + let from_position_exclusive = Position::from(0u64); + let to_position_inclusive = Position::Eof; + let ingester_ids: Vec = vec!["test-ingester-0".into(), "test-ingester-1".into()]; + let (fetch_response_tx, mut fetch_stream) = ServiceStream::new_bounded(5); + + let ingester_pool = IngesterPool::default(); + + let (service_stream_tx_0, service_stream_0) = ServiceStream::new_unbounded(); + let (service_stream_tx_1, service_stream_1) = ServiceStream::new_unbounded(); + + let mut ingester_mock_0 = IngesterServiceClient::mock(); + ingester_mock_0 + .expect_open_fetch_stream() + .return_once(move |request| { + assert_eq!(request.client_id, "test-client"); + assert_eq!(request.index_uid, "test-index:0"); + assert_eq!(request.source_id, "test-source"); + assert_eq!(request.shard_id, 1); + assert_eq!(request.from_position_exclusive(), 0u64); + assert_eq!(request.to_position_inclusive(), Position::Eof); + + Ok(service_stream_0) + }); + let ingester_0: IngesterServiceClient = ingester_mock_0.into(); + + let mut ingester_mock_1 = IngesterServiceClient::mock(); + ingester_mock_1 + .expect_open_fetch_stream() + .return_once(move |request| { + assert_eq!(request.client_id, "test-client"); + assert_eq!(request.index_uid, "test-index:0"); + assert_eq!(request.source_id, "test-source"); + assert_eq!(request.shard_id, 1); + assert_eq!(request.from_position_exclusive(), 1u64); + assert_eq!(request.to_position_inclusive(), Position::Eof); + + Ok(service_stream_1) + }); + let ingester_1: IngesterServiceClient = ingester_mock_1.into(); + + ingester_pool.insert("test-ingester-0".into(), ingester_0); + ingester_pool.insert("test-ingester-1".into(), ingester_1); + + let fetch_response = FetchResponseV2 { + 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)), + }; + service_stream_tx_0.send(Ok(fetch_response)).unwrap(); + + let ingest_error = IngestV2Error::Internal("test-error-0".into()); + service_stream_tx_0.send(Err(ingest_error)).unwrap(); + + let fetch_response = FetchResponseV2 { + 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), + }; + service_stream_tx_1.send(Ok(fetch_response)).unwrap(); + + fault_tolerant_fetch_task( + client_id, + index_uid, + source_id, + shard_id, + from_position_exclusive, + to_position_inclusive, + ingester_ids, + ingester_pool, + fetch_response_tx, + ) + .await; + + let fetch_reponse = timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap() + .unwrap() + .unwrap(); + assert_eq!( + fetch_reponse.from_position_exclusive(), + Position::from(0u64) + ); + assert_eq!(fetch_reponse.to_position_inclusive(), 1u64); + + let fetch_reponse = timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap() + .unwrap() + .unwrap(); + assert_eq!( + fetch_reponse.from_position_exclusive(), + Position::from(1u64) + ); + assert_eq!(fetch_reponse.to_position_inclusive(), Position::Eof); + + assert!(timeout(Duration::from_millis(50), fetch_stream.next()) + .await + .unwrap() + .is_none()); } #[tokio::test] async fn test_multi_fetch_stream() { - let node_id: NodeId = "test-node".into(); + let self_node_id: NodeId = "test-node".into(); let client_id = "test-client".to_string(); let ingester_pool = IngesterPool::default(); - let _multi_fetch_stream = MultiFetchStream::new(node_id, client_id, ingester_pool); + let _multi_fetch_stream = MultiFetchStream::new(self_node_id, client_id, ingester_pool); // TODO: Backport from original branch. } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/gc.rs b/quickwit/quickwit-ingest/src/ingest_v2/gc.rs deleted file mode 100644 index b178867303f..00000000000 --- a/quickwit/quickwit-ingest/src/ingest_v2/gc.rs +++ /dev/null @@ -1,208 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -//! This module holds the logic for removing shards after successful indexing. The -//! ingesters are in charge of monitoring the state of the shards and removing any shard that mets -//! the following criteria: -//! - the shards is closed -//! - its publish position is greater or equal than its replication position -//! - the removal grace period has elapsed -//! -//! Removing a shard consists in: -//! 1. deleting the shard from the metastore -//! 2. removing the shard from the in-memory data structures -//! 3. deleting the associated mrecordlog queue -//! -//! Shard removal should not be confused with shard truncation. Shard truncation is the process of -//! deleting of sets of records from a shard's queue after they have been successfully indexed. - -use std::collections::HashMap; -use std::sync::Arc; -use std::time::Duration; - -use mrecordlog::error::DeleteQueueError; -use once_cell::sync::Lazy; -use quickwit_proto::metastore::{DeleteShardsRequest, DeleteShardsSubrequest}; -use quickwit_proto::{split_queue_id, IndexUid, QueueId, ShardId, SourceId}; -use tokio::sync::{RwLock, Semaphore}; -use tracing::{error, info}; - -use super::ingest_metastore::IngestMetastore; -use super::ingester::IngesterState; - -/// Period of time after which shards are actually deleted. -pub(super) const REMOVAL_GRACE_PERIOD: Duration = if cfg!(any(test, feature = "testsuite")) { - Duration::from_millis(5) -} else { - Duration::from_secs(60 * 10) // 10 minutes -}; - -/// Maximum number of concurrent removal tasks. -const MAX_CONCURRENCY: usize = 3; - -/// Limits the number of concurrent removal tasks. -static MAX_CONCURRENCY_SEMAPHORE: Lazy> = - Lazy::new(|| Arc::new(Semaphore::new(MAX_CONCURRENCY))); - -/// Deletes the shards asynchronously after the grace period has elapsed. -pub(super) fn remove_shards_after( - queues_to_remove: Vec, - grace_period: Duration, - metastore: Arc, - state: Arc>, -) { - if queues_to_remove.is_empty() { - return; - } - let remove_shards_fut = async move { - tokio::time::sleep(grace_period).await; - - let _permit = MAX_CONCURRENCY_SEMAPHORE - .clone() - .acquire_owned() - .await - .expect("semaphore should be open"); - - let mut per_source_shard_ids: HashMap<(IndexUid, SourceId), Vec> = HashMap::new(); - - for queue_id in &queues_to_remove { - let (index_uid, source_id, shard_id) = - split_queue_id(queue_id).expect("queue ID should be well-formed"); - per_source_shard_ids - .entry((index_uid, source_id)) - .or_default() - .push(shard_id); - } - let delete_shards_subrequests = per_source_shard_ids - .into_iter() - .map( - |((index_uid, source_id), shard_ids)| DeleteShardsSubrequest { - index_uid: index_uid.into(), - source_id, - shard_ids, - }, - ) - .collect(); - let delete_shards_request = DeleteShardsRequest { - subrequests: delete_shards_subrequests, - force: false, - }; - if let Err(error) = metastore.delete_shards(delete_shards_request).await { - error!("failed to delete shards: `{}`", error); - return; - } - let mut state_guard = state.write().await; - - for queue_id in &queues_to_remove { - if state_guard.primary_shards.remove(queue_id).is_none() { - state_guard.replica_shards.remove(queue_id); - } - } - for queue_id in &queues_to_remove { - if let Err(DeleteQueueError::IoError(error)) = - state_guard.mrecordlog.delete_queue(queue_id).await - { - error!("failed to delete mrecordlog queue: `{}`", error); - } - } - info!("deleted {} shard(s)", queues_to_remove.len()); - }; - tokio::spawn(remove_shards_fut); -} - -#[cfg(test)] -mod tests { - use mrecordlog::MultiRecordLog; - use quickwit_proto::ingest::ShardState; - use quickwit_proto::metastore::DeleteShardsResponse; - use quickwit_proto::queue_id; - - use super::*; - use crate::ingest_v2::ingest_metastore::MockIngestMetastore; - use crate::ingest_v2::models::{PrimaryShard, ReplicaShard}; - - #[tokio::test] - async fn test_remove_shards() { - let mut mock_metastore = MockIngestMetastore::default(); - mock_metastore - .expect_delete_shards() - .once() - .returning(|request| { - assert_eq!(request.subrequests.len(), 1); - - assert_eq!(request.subrequests[0].index_uid, "test-index:0"); - assert_eq!(request.subrequests[0].source_id, "test-source"); - assert_eq!(request.subrequests[0].shard_ids, [0, 1]); - - let response = DeleteShardsResponse {}; - Ok(response) - }); - let metastore = Arc::new(mock_metastore); - - let queue_id_0 = queue_id("test-index:0", "test-source", 0); - let queue_id_1 = queue_id("test-index:0", "test-source", 1); - - let tempdir = tempfile::tempdir().unwrap(); - let mut mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); - - for queue_id in [&queue_id_0, &queue_id_1] { - mrecordlog.create_queue(queue_id).await.unwrap(); - } - - let mut state = IngesterState { - mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), - replication_streams: HashMap::new(), - replication_tasks: HashMap::new(), - }; - let primary_shard_0 = PrimaryShard::for_test( - Some("test-ingester-1"), - ShardState::Closed, - 12, - 12, - Some(12), - ); - state - .primary_shards - .insert(queue_id_0.clone(), primary_shard_0); - - let replica_shard_1 = ReplicaShard::for_test("test-ingester-1", ShardState::Closed, 42, 42); - state - .replica_shards - .insert(queue_id_1.clone(), replica_shard_1); - - let state = Arc::new(RwLock::new(state)); - - remove_shards_after( - vec![queue_id_0, queue_id_1], - REMOVAL_GRACE_PERIOD, - metastore, - state.clone(), - ); - // Wait for the removal task to run. - tokio::time::sleep(Duration::from_millis(100)).await; - - let state_guard = state.read().await; - assert!(state_guard.primary_shards.is_empty()); - assert!(state_guard.replica_shards.is_empty()); - - assert_eq!(state_guard.mrecordlog.list_queues().count(), 0); - } -} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs deleted file mode 100644 index 570aefa595a..00000000000 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingest_metastore.rs +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -//! [`IngestMetastore`] exposes a small subset of the metastore APIs in a trait defined in this -//! crate instead of depending on the `metastore` crate directly to avoid circular dependencies. It -//! should be removed once we have migrated to the code-generated metastore client. In the meantime, -//! the concrete implementation of this trait lives in the `serve` crate. - -use async_trait::async_trait; -use quickwit_proto::ingest::IngestV2Result; -use quickwit_proto::metastore::{ - CloseShardsRequest, CloseShardsResponse, DeleteShardsRequest, DeleteShardsResponse, -}; - -#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] -#[async_trait] -pub trait IngestMetastore: Send + Sync + 'static { - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> IngestV2Result; - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> IngestV2Result; -} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index c042f0087bf..2ecb940eae9 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -28,7 +28,7 @@ use std::time::Duration; use async_trait::async_trait; use futures::stream::FuturesUnordered; use futures::StreamExt; -use mrecordlog::error::{DeleteQueueError, TruncateError}; +use mrecordlog::error::{CreateQueueError, DeleteQueueError, TruncateError}; use mrecordlog::MultiRecordLog; use quickwit_common::tower::Pool; use quickwit_common::ServiceStream; @@ -40,39 +40,32 @@ use quickwit_proto::ingest::ingester::{ ReplicateSubrequest, SynReplicationMessage, TruncateRequest, TruncateResponse, }; use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; -use quickwit_proto::metastore::{ - CloseShardsFailureKind, CloseShardsRequest, CloseShardsSubrequest, -}; -use quickwit_proto::split_queue_id; -use quickwit_proto::types::{NodeId, QueueId}; -use tokio::sync::{watch, RwLock}; +use quickwit_proto::types::{NodeId, Position, QueueId}; +use tokio::sync::RwLock; use tracing::{error, info}; use super::fetch::FetchTask; -use super::gc::remove_shards_after; -use super::ingest_metastore::IngestMetastore; -use super::models::{Position, PrimaryShard, ReplicaShard, ShardStatus}; -use super::mrecord::MRecord; +use super::models::{IngesterShard, PrimaryShard}; +use super::mrecord::{is_eof_mrecord, MRecord}; use super::replication::{ ReplicationStreamTask, ReplicationStreamTaskHandle, ReplicationTask, ReplicationTaskHandle, SYN_REPLICATION_STREAM_CAPACITY, }; use super::IngesterPool; -use crate::ingest_v2::gc::REMOVAL_GRACE_PERIOD; +use crate::ingest_v2::models::SoloShard; use crate::metrics::INGEST_METRICS; use crate::{FollowerId, LeaderId}; #[derive(Clone)] pub struct Ingester { self_node_id: NodeId, - metastore: Arc, ingester_pool: IngesterPool, state: Arc>, replication_factor: usize, } impl fmt::Debug for Ingester { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { f.debug_struct("Ingester") .field("replication_factor", &self.replication_factor) .finish() @@ -81,30 +74,16 @@ impl fmt::Debug for Ingester { pub(super) struct IngesterState { pub mrecordlog: MultiRecordLog, - pub primary_shards: HashMap, - pub replica_shards: HashMap, + pub shards: HashMap, // Replication stream opened with followers. pub replication_streams: HashMap, // Replication tasks running for each replication stream opened with leaders. pub replication_tasks: HashMap, } -impl IngesterState { - fn find_shard_status_rx(&self, queue_id: &QueueId) -> Option> { - if let Some(shard) = self.primary_shards.get(queue_id) { - return Some(shard.shard_status_rx.clone()); - } - if let Some(shard) = self.replica_shards.get(queue_id) { - return Some(shard.shard_status_rx.clone()); - } - None - } -} - impl Ingester { pub async fn try_new( self_node_id: NodeId, - metastore: Arc, ingester_pool: Pool, wal_dir_path: &Path, replication_factor: usize, @@ -118,14 +97,12 @@ impl Ingester { let inner = IngesterState { mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), + shards: HashMap::new(), replication_streams: HashMap::new(), replication_tasks: HashMap::new(), }; let mut ingester = Self { self_node_id, - metastore, ingester_pool, state: Arc::new(RwLock::new(inner)), replication_factor, @@ -141,143 +118,55 @@ impl Ingester { } async fn init(&mut self) -> IngestV2Result<()> { - let mut per_queue_positions: HashMap> = HashMap::new(); - let state_guard = self.state.read().await; - - for queue_id in state_guard.mrecordlog.list_queues() { - let current_position = state_guard - .mrecordlog - .current_position(queue_id) - .expect("queue should exist"); - per_queue_positions.insert(queue_id.to_string(), current_position); - } - let mut subrequests = Vec::new(); - - for (queue_id, current_position) in &per_queue_positions { - let (index_uid, source_id, shard_id) = - split_queue_id(queue_id).expect("queue ID should be well-formed"); - - let subrequest = CloseShardsSubrequest { - index_uid: index_uid.into(), - source_id, - shard_id, - shard_state: ShardState::Closed as i32, - replication_position_inclusive: *current_position, - }; - subrequests.push(subrequest); - } - if subrequests.is_empty() { - return Ok(()); - } - drop(state_guard); - - let close_shards_request = CloseShardsRequest { subrequests }; - let close_shards_response = self.metastore.close_shards(close_shards_request).await?; - info!("closed {} shard(s)", close_shards_response.successes.len()); - let mut state_guard = self.state.write().await; - // Keep track of the queues that can be safely deleted. - let mut queues_to_remove: Vec = Vec::new(); - - for success in close_shards_response.successes { - let queue_id = success.queue_id(); - let publish_position_inclusive: Position = success.publish_position_inclusive.into(); - - if let Some(truncate_position) = publish_position_inclusive.offset() { - state_guard - .mrecordlog - .truncate(&queue_id, truncate_position) - .await - .expect("queue should exist"); - } - let current_position: Position = (*per_queue_positions - .get(&queue_id) - .expect("queue should exist")) - .into(); - let shard_status = ShardStatus { - shard_state: ShardState::Closed, - publish_position_inclusive, - replication_position_inclusive: current_position, - }; - let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); + let queue_ids: Vec = state_guard + .mrecordlog + .list_queues() + .map(|queue_id| queue_id.to_string()) + .collect(); - if publish_position_inclusive >= current_position { - queues_to_remove.push(queue_id.clone()) - } - if success.leader_id == self.self_node_id { - let primary_shard = PrimaryShard { - follower_id_opt: success.follower_id.map(|follower_id| follower_id.into()), - shard_state: ShardState::Closed, - publish_position_inclusive, - primary_position_inclusive: current_position, - replica_position_inclusive_opt: None, - shard_status_tx, - shard_status_rx, - }; - state_guard.primary_shards.insert(queue_id, primary_shard); - } else { - let replica_shard = ReplicaShard { - _leader_id: success.leader_id.into(), - shard_state: ShardState::Closed, - publish_position_inclusive, - replica_position_inclusive: current_position, - shard_status_tx, - shard_status_rx, - }; - state_guard.replica_shards.insert(queue_id, replica_shard); - } - } - for failure in close_shards_response.failures { - if failure.failure_kind() == CloseShardsFailureKind::NotFound { - let queue_id = failure.queue_id(); + for queue_id in queue_ids { + append_eof_record_if_necessary(&mut state_guard.mrecordlog, &queue_id).await; - if let Err(DeleteQueueError::IoError(error)) = - state_guard.mrecordlog.delete_queue(&queue_id).await - { - error!("failed to delete mrecordlog queue: {}", error); - } - } + let solo_shard = SoloShard::new(ShardState::Closed, Position::Eof); + let shard = IngesterShard::Solo(solo_shard); + state_guard.shards.insert(queue_id, shard); } - remove_shards_after( - queues_to_remove, - REMOVAL_GRACE_PERIOD, - self.metastore.clone(), - self.state.clone(), - ); Ok(()) } - async fn init_primary_shard<'a>( + async fn create_shard<'a>( &self, state: &'a mut IngesterState, queue_id: &QueueId, leader_id: &NodeId, follower_id_opt: Option<&NodeId>, - ) -> IngestV2Result<&'a PrimaryShard> { - if !state.mrecordlog.queue_exists(queue_id) { - state.mrecordlog.create_queue(queue_id).await.expect("TODO"); // IO error, what to do? - } else { - // TODO: Recover last position from mrecordlog and take it from there. - } - if let Some(follower_id) = follower_id_opt { + ) -> IngestV2Result<&'a IngesterShard> { + match state.mrecordlog.create_queue(queue_id).await { + Ok(_) => {} + Err(CreateQueueError::AlreadyExists) => panic!("queue should not exist"), + Err(CreateQueueError::IoError(io_error)) => { + // TODO: Close all shards and set readiness to false. + error!( + "failed to create mrecordlog queue `{}`: {}", + queue_id, io_error + ); + return Err(IngestV2Error::IngesterUnavailable { + ingester_id: leader_id.clone(), + }); + } + }; + let shard = if let Some(follower_id) = follower_id_opt { self.init_replication_stream(state, leader_id, follower_id) .await?; - } - let replica_position_inclusive_opt = follower_id_opt.map(|_| Position::default()); - let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); - - let primary_shard = PrimaryShard { - follower_id_opt: follower_id_opt.cloned(), - shard_state: ShardState::Open, - publish_position_inclusive: Position::default(), - primary_position_inclusive: Position::default(), - replica_position_inclusive_opt, - shard_status_tx, - shard_status_rx, + let primary_shard = PrimaryShard::new(follower_id.clone()); + IngesterShard::Primary(primary_shard) + } else { + IngesterShard::Solo(SoloShard::default()) }; - let entry = state.primary_shards.entry(queue_id.clone()); - Ok(entry.or_insert(primary_shard)) + let entry = state.shards.entry(queue_id.clone()); + Ok(entry.or_insert(shard)) } async fn init_replication_stream( @@ -287,7 +176,7 @@ impl Ingester { follower_id: &NodeId, ) -> IngestV2Result<()> { let Entry::Vacant(entry) = state.replication_streams.entry(follower_id.clone()) else { - // The replication client is already initialized. Nothing to do! + // A replication stream with this follower is already opened. return Ok(()); }; let open_request = OpenReplicationStreamRequest { @@ -353,20 +242,22 @@ impl IngesterService for Ingester { for subrequest in persist_request.subrequests { let queue_id = subrequest.queue_id(); - let follower_id: Option = subrequest.follower_id.map(Into::into); - let primary_shard = - if let Some(primary_shard) = state_guard.primary_shards.get(&queue_id) { - primary_shard - } else { - self.init_primary_shard( - &mut state_guard, - &queue_id, - &leader_id, - follower_id.as_ref(), - ) - .await? - }; - if primary_shard.shard_state.is_closed() { + let follower_id_opt: Option = subrequest.follower_id.map(Into::into); + let shard = if let Some(shard) = state_guard.shards.get_mut(&queue_id) { + shard + } else { + self.create_shard( + &mut state_guard, + &queue_id, + &leader_id, + follower_id_opt.as_ref(), + ) + .await + .expect("TODO") + }; + let from_position_exclusive = shard.replication_position_inclusive(); + + if shard.is_closed() { let persist_failure = PersistFailure { index_uid: subrequest.index_uid, source_id: subrequest.source_id, @@ -376,19 +267,11 @@ impl IngesterService for Ingester { persist_failures.push(persist_failure); continue; } - let from_position_inclusive = primary_shard.primary_position_inclusive; + let doc_batch = subrequest + .doc_batch + .expect("router should not send empty persist subrequests"); - let Some(doc_batch) = subrequest.doc_batch else { - let persist_success = PersistSuccess { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - replication_position_inclusive: from_position_inclusive.offset(), - }; - persist_successes.push(persist_success); - continue; - }; - let primary_position_inclusive = if force_commit { + let current_position_inclusive: Position = if force_commit { let encoded_mrecords = doc_batch .docs() .map(|doc| MRecord::Doc(doc).encode()) @@ -405,7 +288,8 @@ impl IngesterService for Ingester { .append_records(&queue_id, None, encoded_mrecords) .await .expect("TODO") // TODO: Io error, close shard? - }; + } + .into(); let batch_num_bytes = doc_batch.num_bytes() as u64; let batch_num_docs = doc_batch.num_docs() as u64; @@ -413,17 +297,18 @@ impl IngesterService for Ingester { INGEST_METRICS.ingested_num_docs.inc_by(batch_num_docs); state_guard - .primary_shards + .shards .get_mut(&queue_id) .expect("primary shard should exist") - .set_primary_position_inclusive(primary_position_inclusive); + .set_replication_position_inclusive(current_position_inclusive.clone()); - if let Some(follower_id) = follower_id { + if let Some(follower_id) = follower_id_opt { let replicate_subrequest = ReplicateSubrequest { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - from_position_exclusive: from_position_inclusive.offset(), + from_position_exclusive: Some(from_position_exclusive), + to_position_inclusive: Some(current_position_inclusive), doc_batch: Some(doc_batch), }; replicate_subrequests @@ -435,7 +320,7 @@ impl IngesterService for Ingester { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - replication_position_inclusive: primary_position_inclusive, + replication_position_inclusive: Some(current_position_inclusive), }; persist_successes.push(persist_success); } @@ -487,21 +372,16 @@ impl IngesterService for Ingester { index_uid: replicate_success.index_uid, source_id: replicate_success.source_id, shard_id: replicate_success.shard_id, - replication_position_inclusive: replicate_success.replica_position_inclusive, + replication_position_inclusive: replicate_success + .replication_position_inclusive, }; persist_successes.push(persist_success); } } - let mut state_guard = self.state.write().await; + let _state_guard = self.state.write().await; for persist_success in &persist_successes { - let queue_id = persist_success.queue_id(); - - state_guard - .primary_shards - .get_mut(&queue_id) - .expect("TODO") - .set_replica_position_inclusive(persist_success.replication_position_inclusive); + let _queue_id = persist_success.queue_id(); } let leader_id = self.self_node_id.to_string(); let persist_response = PersistResponse { @@ -562,16 +442,18 @@ impl IngesterService for Ingester { open_fetch_stream_request: OpenFetchStreamRequest, ) -> IngestV2Result>> { let queue_id = open_fetch_stream_request.queue_id(); - let shard_status_rx = self + let new_records_rx = self .state .read() .await - .find_shard_status_rx(&queue_id) - .ok_or_else(|| IngestV2Error::Internal("shard not found".to_string()))?; + .shards + .get(&queue_id) + .ok_or_else(|| IngestV2Error::Internal("shard not found".to_string()))? + .new_records_rx(); let (service_stream, _fetch_task_handle) = FetchTask::spawn( open_fetch_stream_request, self.state.clone(), - shard_status_rx, + new_records_rx, FetchTask::DEFAULT_BATCH_NUM_BYTES, ); Ok(service_stream) @@ -607,56 +489,67 @@ impl IngesterService for Ingester { self.self_node_id, truncate_request.ingester_id, ))); } - let mut queues_to_remove: Vec = Vec::new(); let mut state_guard = self.state.write().await; for subrequest in truncate_request.subrequests { let queue_id = subrequest.queue_id(); + let to_position_inclusive = subrequest.to_position_inclusive(); - match state_guard - .mrecordlog - .truncate(&queue_id, subrequest.to_position_inclusive) - .await - { - Ok(_) | Err(TruncateError::MissingQueue(_)) => {} - Err(error) => { - error!("failed to truncate queue `{}`: {}", queue_id, error); - continue; - } - } - if let Some(primary_shard) = state_guard.primary_shards.get_mut(&queue_id) { - primary_shard.set_publish_position_inclusive(subrequest.to_position_inclusive); - - if primary_shard.is_removable() { - queues_to_remove.push(queue_id.clone()); + if let Some(to_offset_inclusive) = to_position_inclusive.as_u64() { + match state_guard + .mrecordlog + .truncate(&queue_id, to_offset_inclusive) + .await + { + Ok(_) | Err(TruncateError::MissingQueue(_)) => {} + Err(error) => { + error!("failed to truncate queue `{}`: {}", queue_id, error); + } } - continue; - } - if let Some(replica_shard) = state_guard.replica_shards.get_mut(&queue_id) { - replica_shard.set_publish_position_inclusive(subrequest.to_position_inclusive); - - if replica_shard.is_removable() { - queues_to_remove.push(queue_id.clone()); + } else if to_position_inclusive == Position::Eof { + match state_guard.mrecordlog.delete_queue(&queue_id).await { + Ok(_) | Err(DeleteQueueError::MissingQueue(_)) => {} + Err(error) => { + error!("failed to delete queue `{}`: {}", queue_id, error); + } } - } + state_guard.shards.remove(&queue_id); + }; } - drop(state_guard); - - remove_shards_after( - queues_to_remove, - REMOVAL_GRACE_PERIOD, - self.metastore.clone(), - self.state.clone(), - ); let truncate_response = TruncateResponse {}; Ok(truncate_response) } } +/// Appends an EOF record to the queue if the it is empty or the last record is not an EOF +/// record. +/// +/// # Panics +/// +/// Panics if the queue does not exist. +async fn append_eof_record_if_necessary(mrecordlog: &mut MultiRecordLog, queue_id: &QueueId) { + let mut should_append_eof_record = true; + + if let Some(current_position) = mrecordlog.current_position(queue_id).expect("TODO") { + let mrecords = mrecordlog + .range(queue_id, current_position..) + .expect("TODO"); + + if let Some((_, last_mecord)) = mrecords.last() { + should_append_eof_record = !is_eof_mrecord(&last_mecord); + } + } + if should_append_eof_record { + mrecordlog + .append_record(queue_id, None, MRecord::Eof.encode()) + .await + .expect("TODO"); + } +} + #[cfg(test)] mod tests { use std::net::SocketAddr; - use std::time::Duration; use bytes::Bytes; use quickwit_proto::ingest::ingester::{ @@ -664,103 +557,21 @@ mod tests { TruncateSubrequest, }; use quickwit_proto::ingest::DocBatchV2; - use quickwit_proto::metastore::{ - CloseShardsFailure, CloseShardsFailureKind, CloseShardsResponse, CloseShardsSuccess, - DeleteShardsResponse, - }; use quickwit_proto::types::queue_id; use tonic::transport::{Endpoint, Server}; use super::*; - use crate::ingest_v2::ingest_metastore::MockIngestMetastore; - use crate::ingest_v2::test_utils::{ - MultiRecordLogTestExt, PrimaryShardTestExt, ReplicaShardTestExt, - }; - - const NONE_REPLICA_POSITION: Option = None; + use crate::ingest_v2::test_utils::{IngesterShardTestExt, MultiRecordLogTestExt}; #[tokio::test] async fn test_ingester_init() { let tempdir = tempfile::tempdir().unwrap(); let self_node_id: NodeId = "test-ingester-0".into(); - let mut mock_metastore = MockIngestMetastore::default(); - mock_metastore - .expect_close_shards() - .once() - .returning(|request| { - assert_eq!(request.subrequests.len(), 4); - let mut subrequests = request.subrequests; - subrequests.sort_unstable_by(|left, right| left.shard_id.cmp(&right.shard_id)); - - for (i, subrequest) in subrequests.iter().enumerate() { - assert_eq!(subrequest.index_uid, "test-index:0"); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_id, i as u64 + 1); - assert_eq!(subrequest.shard_state(), ShardState::Closed); - } - assert!(subrequests[0].replication_position_inclusive.is_none()); - assert_eq!(subrequests[1].replication_position_inclusive, Some(1)); - assert_eq!(subrequests[2].replication_position_inclusive, Some(1)); - assert_eq!(subrequests[3].replication_position_inclusive, Some(1)); - - let response = CloseShardsResponse { - successes: vec![ - CloseShardsSuccess { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 2, - leader_id: "test-ingester-0".to_string(), - follower_id: Some("test-ingester-1".to_string()), - publish_position_inclusive: "1".to_string(), - }, - CloseShardsSuccess { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 3, - leader_id: "test-ingester-1".to_string(), - follower_id: Some("test-ingester-0".to_string()), - publish_position_inclusive: "0".to_string(), - }, - CloseShardsSuccess { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 4, - leader_id: "test-ingester-0".to_string(), - follower_id: Some("test-ingester-1".to_string()), - publish_position_inclusive: "".to_string(), - }, - ], - failures: vec![CloseShardsFailure { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 1, - failure_kind: CloseShardsFailureKind::NotFound as i32, - failure_message: "shard not found".to_string(), - }], - }; - Ok(response) - }); - mock_metastore - .expect_delete_shards() - .once() - .returning(|request| { - assert_eq!(request.subrequests.len(), 1); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.index_uid, "test-index:0"); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, [2]); - - let response = DeleteShardsResponse {}; - Ok(response) - }); - let metastore = Arc::new(mock_metastore); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 2; let mut ingester = Ingester::try_new( self_node_id.clone(), - metastore, ingester_pool, wal_dir_path, replication_factor, @@ -770,101 +581,98 @@ mod tests { let mut state_guard = ingester.state.write().await; - let queue_ids: Vec = (1..=4) - .map(|shard_id| queue_id("test-index:0", "test-source", shard_id)) - .collect(); + let queue_id_01 = queue_id("test-index:0", "test-source", 1); + state_guard + .mrecordlog + .create_queue(&queue_id_01) + .await + .unwrap(); + + let records = [MRecord::new_doc("test-doc-foo").encode()].into_iter(); - for queue_id in &queue_ids { - state_guard.mrecordlog.create_queue(queue_id).await.unwrap(); - } - let records = [ - MRecord::new_doc("test-doc-200").encode(), - MRecord::new_doc("test-doc-201").encode(), - ]; state_guard .mrecordlog - .append_records(&queue_ids[1], None, records.into_iter()) + .append_records(&queue_id_01, None, records) .await .unwrap(); - let records = [ - MRecord::new_doc("test-doc-300").encode(), - MRecord::new_doc("test-doc-301").encode(), - ]; + state_guard .mrecordlog - .append_records(&queue_ids[2], None, records.into_iter()) + .truncate(&queue_id_01, 0) .await .unwrap(); - let records = [ - MRecord::new_doc("test-doc-400").encode(), - MRecord::new_doc("test-doc-401").encode(), - ]; + + let queue_id_02 = queue_id("test-index:0", "test-source", 2); + state_guard + .mrecordlog + .create_queue(&queue_id_02) + .await + .unwrap(); + + let records = [MRecord::new_doc("test-doc-foo").encode()].into_iter(); + state_guard .mrecordlog - .append_records(&queue_ids[3], None, records.into_iter()) + .append_records(&queue_id_02, None, records) .await .unwrap(); + + let queue_id_03 = queue_id("test-index:0", "test-source", 3); + state_guard + .mrecordlog + .create_queue(&queue_id_03) + .await + .unwrap(); + drop(state_guard); ingester.init().await.unwrap(); - let state_guard = ingester.state.read().await; - assert!(!state_guard.mrecordlog.queue_exists(&queue_ids[0])); - drop(state_guard); + // It should only append EOF records if necessary. + ingester.init().await.unwrap(); let state_guard = ingester.state.read().await; - assert_eq!(state_guard.primary_shards.len(), 2); - assert_eq!(state_guard.replica_shards.len(), 1); + assert_eq!(state_guard.shards.len(), 3); - let primary_shard_2 = state_guard.primary_shards.get(&queue_ids[1]).unwrap(); - assert_eq!( - primary_shard_2.publish_position_inclusive, - Position::Offset(1) - ); - assert!(primary_shard_2.shard_state.is_closed()); - primary_shard_2.assert_positions(Some(1), NONE_REPLICA_POSITION); + let 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); - let primary_shard_4 = state_guard.primary_shards.get(&queue_ids[3]).unwrap(); - assert_eq!( - primary_shard_4.publish_position_inclusive, - Position::Beginning - ); - assert!(primary_shard_4.shard_state.is_closed()); - primary_shard_2.assert_positions(Some(1), NONE_REPLICA_POSITION); + state_guard + .mrecordlog + .assert_records_eq(&queue_id_01, .., &[(1, "\0\x02")]); - let replica_shard_3 = state_guard.replica_shards.get(&queue_ids[2]).unwrap(); - assert_eq!( - replica_shard_3.publish_position_inclusive, - Position::Offset(0), - ); - assert_eq!( - replica_shard_3.replica_position_inclusive, - Position::Offset(1), - ); - assert!(replica_shard_3.shard_state.is_closed()); + 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); - drop(state_guard); + state_guard.mrecordlog.assert_records_eq( + &queue_id_02, + .., + &[(0, "\0\0test-doc-foo"), (1, "\0\x02")], + ); - // Wait for the removal task to run. - tokio::time::sleep(Duration::from_millis(100)).await; + 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); - let state_guard = ingester.state.read().await; - assert_eq!(state_guard.primary_shards.len(), 1); - assert!(!state_guard.primary_shards.contains_key(&queue_ids[1])); - assert!(!state_guard.mrecordlog.queue_exists(&queue_ids[1])); + state_guard + .mrecordlog + .assert_records_eq(&queue_id_03, .., &[(0, "\0\x02")]); } #[tokio::test] async fn test_ingester_persist() { let tempdir = tempfile::tempdir().unwrap(); let self_node_id: NodeId = "test-ingester-0".into(); - let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( self_node_id.clone(), - metastore, ingester_pool, wal_dir_path, replication_factor, @@ -876,13 +684,6 @@ mod tests { leader_id: self_node_id.to_string(), commit_type: CommitTypeV2::Force as i32, subrequests: vec![ - PersistSubrequest { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 0, - follower_id: None, - doc_batch: None, - }, PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), @@ -893,49 +694,42 @@ mod tests { PersistSubrequest { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-100", "test-doc-101"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), }, ], }; ingester.persist(persist_request).await.unwrap(); let state_guard = ingester.state.read().await; - assert_eq!(state_guard.primary_shards.len(), 3); - - let queue_id_00 = queue_id("test-index:0", "test-source", 0); - let primary_shard_00 = state_guard.primary_shards.get(&queue_id_00).unwrap(); - primary_shard_00.assert_positions(None, NONE_REPLICA_POSITION); - primary_shard_00.assert_is_open(None); - - state_guard - .mrecordlog - .assert_records_eq(&queue_id_00, .., &[]); + assert_eq!(state_guard.shards.len(), 2); let queue_id_01 = queue_id("test-index:0", "test-source", 1); - let primary_shard_01 = state_guard.primary_shards.get(&queue_id_01).unwrap(); - primary_shard_01.assert_positions(1, NONE_REPLICA_POSITION); - primary_shard_01.assert_is_open(1); + let solo_shard_01 = state_guard.shards.get(&queue_id_01).unwrap(); + solo_shard_01.assert_is_solo(); + solo_shard_01.assert_is_open(); + solo_shard_01.assert_replication_position(1u64); state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010"), (1, "\0\u{1}")], + &[(0, "\0\0test-doc-010"), (1, "\0\x01")], ); - let queue_id_10 = queue_id("test-index:1", "test-source", 0); - let primary_shard_10 = state_guard.primary_shards.get(&queue_id_10).unwrap(); - primary_shard_10.assert_positions(2, NONE_REPLICA_POSITION); - primary_shard_10.assert_is_open(2); + let queue_id_11 = queue_id("test-index:1", "test-source", 1); + let solo_shard_11 = state_guard.shards.get(&queue_id_11).unwrap(); + solo_shard_11.assert_is_solo(); + solo_shard_11.assert_is_open(); + solo_shard_11.assert_replication_position(2u64); state_guard.mrecordlog.assert_records_eq( - &queue_id_10, + &queue_id_11, .., &[ - (0, "\0\0test-doc-100"), - (1, "\0\0test-doc-101"), - (2, "\0\u{1}"), + (0, "\0\0test-doc-110"), + (1, "\0\0test-doc-111"), + (2, "\0\x01"), ], ); } @@ -944,13 +738,11 @@ mod tests { async fn test_ingester_open_replication_stream() { let tempdir = tempfile::tempdir().unwrap(); let self_node_id: NodeId = "test-follower".into(); - let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( self_node_id.clone(), - metastore, ingester_pool, wal_dir_path, replication_factor, @@ -987,13 +779,11 @@ mod tests { async fn test_ingester_persist_replicate() { let tempdir = tempfile::tempdir().unwrap(); let leader_id: NodeId = "test-leader".into(); - let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 2; let mut leader = Ingester::try_new( leader_id.clone(), - metastore.clone(), ingester_pool.clone(), wal_dir_path, replication_factor, @@ -1007,7 +797,6 @@ mod tests { let replication_factor = 2; let follower = Ingester::try_new( follower_id.clone(), - metastore, ingester_pool.clone(), wal_dir_path, replication_factor, @@ -1022,15 +811,8 @@ mod tests { let persist_request = PersistRequest { leader_id: "test-leader".to_string(), - commit_type: CommitTypeV2::Auto as i32, + commit_type: CommitTypeV2::Force as i32, subrequests: vec![ - PersistSubrequest { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 0, - follower_id: Some(follower_id.to_string()), - doc_batch: None, - }, PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), @@ -1041,49 +823,75 @@ mod tests { PersistSubrequest { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, follower_id: Some(follower_id.to_string()), - doc_batch: Some(DocBatchV2::for_test(["test-doc-100", "test-doc-101"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), }, ], }; let persist_response = leader.persist(persist_request).await.unwrap(); assert_eq!(persist_response.leader_id, "test-leader"); - assert_eq!(persist_response.successes.len(), 3); + assert_eq!(persist_response.successes.len(), 2); assert_eq!(persist_response.failures.len(), 0); let leader_state_guard = leader.state.read().await; - assert_eq!(leader_state_guard.primary_shards.len(), 3); - - let queue_id_00 = queue_id("test-index:0", "test-source", 0); - let primary_shard_00 = leader_state_guard.primary_shards.get(&queue_id_00).unwrap(); - primary_shard_00.assert_positions(None, Some(None)); - primary_shard_00.assert_is_open(None); - - leader_state_guard - .mrecordlog - .assert_records_eq(&queue_id_00, .., &[]); + assert_eq!(leader_state_guard.shards.len(), 2); let queue_id_01 = queue_id("test-index:0", "test-source", 1); - let primary_shard_01 = leader_state_guard.primary_shards.get(&queue_id_01).unwrap(); - primary_shard_01.assert_positions(0, Some(0)); - primary_shard_01.assert_is_open(0); + 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); leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010")], + &[(0, "\0\0test-doc-010"), (1, "\0\x01")], ); - let queue_id_10 = queue_id("test-index:1", "test-source", 0); - let primary_shard_10 = leader_state_guard.primary_shards.get(&queue_id_10).unwrap(); - primary_shard_10.assert_positions(1, Some(1)); - primary_shard_10.assert_is_open(1); + let queue_id_11 = queue_id("test-index:1", "test-source", 1); + let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); + primary_shard_11.assert_is_primary(); + primary_shard_11.assert_is_open(); + primary_shard_11.assert_replication_position(2u64); leader_state_guard.mrecordlog.assert_records_eq( - &queue_id_10, + &queue_id_11, + .., + &[ + (0, "\0\0test-doc-110"), + (1, "\0\0test-doc-111"), + (2, "\0\x01"), + ], + ); + + let follower_state_guard = follower.state.read().await; + assert_eq!(follower_state_guard.shards.len(), 2); + + 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); + + follower_state_guard.mrecordlog.assert_records_eq( + &queue_id_01, .., - &[(0, "\0\0test-doc-100"), (1, "\0\0test-doc-101")], + &[(0, "\0\0test-doc-010"), (1, "\0\x01")], + ); + + 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); + + follower_state_guard.mrecordlog.assert_records_eq( + &queue_id_11, + .., + &[ + (0, "\0\0test-doc-110"), + (1, "\0\0test-doc-111"), + (2, "\0\x01"), + ], ); } @@ -1091,13 +899,11 @@ mod tests { async fn test_ingester_persist_replicate_grpc() { let tempdir = tempfile::tempdir().unwrap(); let leader_id: NodeId = "test-leader".into(); - let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 2; let mut leader = Ingester::try_new( leader_id.clone(), - metastore.clone(), ingester_pool.clone(), wal_dir_path, replication_factor, @@ -1125,7 +931,6 @@ mod tests { let replication_factor = 2; let follower = Ingester::try_new( follower_id.clone(), - metastore, ingester_pool.clone(), wal_dir_path, replication_factor, @@ -1158,13 +963,6 @@ mod tests { leader_id: "test-leader".to_string(), commit_type: CommitTypeV2::Auto as i32, subrequests: vec![ - PersistSubrequest { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 0, - follower_id: Some(follower_id.to_string()), - doc_batch: None, - }, PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), @@ -1175,98 +973,79 @@ mod tests { PersistSubrequest { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, follower_id: Some(follower_id.to_string()), - doc_batch: Some(DocBatchV2::for_test(["test-doc-100", "test-doc-101"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-110", "test-doc-111"])), }, ], }; let persist_response = leader.persist(persist_request).await.unwrap(); assert_eq!(persist_response.leader_id, "test-leader"); - assert_eq!(persist_response.successes.len(), 3); + assert_eq!(persist_response.successes.len(), 2); assert_eq!(persist_response.failures.len(), 0); - let queue_id_00 = queue_id("test-index:0", "test-source", 0); - let leader_state_guard = leader.state.read().await; - leader_state_guard - .mrecordlog - .assert_records_eq(&queue_id_00, .., &[]); - - let primary_shard = leader_state_guard.primary_shards.get(&queue_id_00).unwrap(); - primary_shard.assert_positions(None, Some(None)); - primary_shard.assert_is_open(None); - - let follower_state_guard = follower.state.read().await; - assert!(!follower_state_guard.mrecordlog.queue_exists(&queue_id_00)); - - assert!(!follower_state_guard - .replica_shards - .contains_key(&queue_id_00)); + assert_eq!(leader_state_guard.shards.len(), 2); let queue_id_01 = queue_id("test-index:0", "test-source", 1); + let primary_shard_01 = leader_state_guard.shards.get(&queue_id_01).unwrap(); + primary_shard_01.assert_is_primary(); + primary_shard_01.assert_is_open(); + primary_shard_01.assert_replication_position(0u64); - let leader_state_guard = leader.state.read().await; leader_state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., &[(0, "\0\0test-doc-010")], ); - let primary_shard = leader_state_guard.primary_shards.get(&queue_id_01).unwrap(); - primary_shard.assert_positions(0, Some(0)); - primary_shard.assert_is_open(0); + let queue_id_11 = queue_id("test-index:1", "test-source", 1); + let primary_shard_11 = leader_state_guard.shards.get(&queue_id_11).unwrap(); + primary_shard_11.assert_is_primary(); + primary_shard_11.assert_is_open(); + primary_shard_11.assert_replication_position(1u64); - follower_state_guard.mrecordlog.assert_records_eq( - &queue_id_01, + leader_state_guard.mrecordlog.assert_records_eq( + &queue_id_11, .., - &[(0, "\0\0test-doc-010")], + &[(0, "\0\0test-doc-110"), (1, "\0\0test-doc-111")], ); - let replica_shard = follower_state_guard - .replica_shards - .get(&queue_id_01) - .unwrap(); - replica_shard.assert_position(0); - replica_shard.assert_is_open(0); + let follower_state_guard = follower.state.read().await; + assert_eq!(follower_state_guard.shards.len(), 2); - let queue_id_10 = queue_id("test-index:1", "test-source", 0); + 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); - leader_state_guard.mrecordlog.assert_records_eq( - &queue_id_10, + follower_state_guard.mrecordlog.assert_records_eq( + &queue_id_01, .., - &[(0, "\0\0test-doc-100"), (1, "\0\0test-doc-101")], + &[(0, "\0\0test-doc-010")], ); - let primary_shard = leader_state_guard.primary_shards.get(&queue_id_10).unwrap(); - primary_shard.assert_positions(1, Some(1)); - primary_shard.assert_is_open(1); + 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); follower_state_guard.mrecordlog.assert_records_eq( - &queue_id_10, + &queue_id_11, .., - &[(0, "\0\0test-doc-100"), (1, "\0\0test-doc-101")], + &[(0, "\0\0test-doc-110"), (1, "\0\0test-doc-111")], ); - - let replica_shard = follower_state_guard - .replica_shards - .get(&queue_id_10) - .unwrap(); - replica_shard.assert_position(1); - replica_shard.assert_is_open(1); } #[tokio::test] async fn test_ingester_open_fetch_stream() { let tempdir = tempfile::tempdir().unwrap(); let self_node_id: NodeId = "test-ingester-0".into(); - let metastore = Arc::new(MockIngestMetastore::default()); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( self_node_id.clone(), - metastore, ingester_pool, wal_dir_path, replication_factor, @@ -1281,27 +1060,28 @@ mod tests { PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-000"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), }, PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: 2, follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-010"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-020"])), }, ], }; ingester.persist(persist_request).await.unwrap(); let client_id = "test-client".to_string(); + let open_fetch_stream_request = OpenFetchStreamRequest { client_id: client_id.clone(), index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, from_position_exclusive: None, to_position_inclusive: None, }; @@ -1309,14 +1089,20 @@ 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 + ); + assert_eq!(fetch_response.to_position_inclusive(), Position::from(0u64)); + let mrecord_batch = fetch_response.mrecord_batch.unwrap(); assert_eq!( mrecord_batch.mrecord_buffer, - Bytes::from_static(b"\0\0test-doc-000") + Bytes::from_static(b"\0\0test-doc-010") ); assert_eq!(mrecord_batch.mrecord_lengths, [14]); - assert_eq!(fetch_response.from_position_inclusive, 0); let persist_request = PersistRequest { leader_id: self_node_id.to_string(), @@ -1324,49 +1110,37 @@ mod tests { subrequests: vec![PersistSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, follower_id: None, - doc_batch: Some(DocBatchV2::for_test(["test-doc-001", "test-doc-002"])), + doc_batch: Some(DocBatchV2::for_test(["test-doc-011", "test-doc-012"])), }], }; ingester.persist(persist_request).await.unwrap(); let fetch_response = fetch_stream.next().await.unwrap().unwrap(); + assert_eq!( + fetch_response.from_position_exclusive(), + Position::from(0u64) + ); + assert_eq!(fetch_response.to_position_inclusive(), Position::from(2u64)); + let mrecord_batch = fetch_response.mrecord_batch.unwrap(); assert_eq!( mrecord_batch.mrecord_buffer, - Bytes::from_static(b"\0\0test-doc-001\0\0test-doc-002") + Bytes::from_static(b"\0\0test-doc-011\0\0test-doc-012") ); assert_eq!(mrecord_batch.mrecord_lengths, [14, 14]); - assert_eq!(fetch_response.from_position_inclusive, 1); } #[tokio::test] async fn test_ingester_truncate() { let tempdir = tempfile::tempdir().unwrap(); let self_node_id: NodeId = "test-ingester-0".into(); - let mut mock_metastore = MockIngestMetastore::default(); - mock_metastore - .expect_delete_shards() - .once() - .returning(|request| { - assert_eq!(request.subrequests.len(), 1); - - let subrequest = &request.subrequests[0]; - assert_eq!(subrequest.index_uid, "test-index:0"); - assert_eq!(subrequest.source_id, "test-source"); - assert_eq!(subrequest.shard_ids, [2]); - - let response = DeleteShardsResponse {}; - Ok(response) - }); - let metastore = Arc::new(mock_metastore); let ingester_pool = IngesterPool::default(); let wal_dir_path = tempdir.path(); let replication_factor = 1; let mut ingester = Ingester::try_new( self_node_id.clone(), - metastore, ingester_pool, wal_dir_path, replication_factor, @@ -1379,22 +1153,17 @@ mod tests { let mut state_guard = ingester.state.write().await; ingester - .init_primary_shard(&mut state_guard, &queue_id_01, &self_node_id, None) + .create_shard(&mut state_guard, &queue_id_01, &self_node_id, None) .await .unwrap(); ingester - .init_primary_shard(&mut state_guard, &queue_id_02, &self_node_id, None) + .create_shard(&mut state_guard, &queue_id_02, &self_node_id, None) .await .unwrap(); - state_guard - .primary_shards - .get_mut(&queue_id_02) - .unwrap() - .shard_state = ShardState::Closed; let records = [ - Bytes::from_static(b"test-doc-000"), - Bytes::from_static(b"test-doc-001"), + MRecord::new_doc("test-doc-010").encode(), + MRecord::new_doc("test-doc-011").encode(), ] .into_iter(); @@ -1404,11 +1173,7 @@ mod tests { .await .unwrap(); - let records = [ - Bytes::from_static(b"test-doc-010"), - Bytes::from_static(b"test-doc-011"), - ] - .into_iter(); + let records = [MRecord::new_doc("test-doc-020").encode()].into_iter(); state_guard .mrecordlog @@ -1425,60 +1190,31 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - to_position_inclusive: 0, + to_position_inclusive: Some(Position::from(0u64)), }, TruncateSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 2, - to_position_inclusive: 1, + to_position_inclusive: Some(Position::Eof), }, TruncateSubrequest { index_uid: "test-index:1337".to_string(), source_id: "test-source".to_string(), shard_id: 1, - to_position_inclusive: 1337, + to_position_inclusive: Some(Position::from(1337u64)), }, ], }; ingester.truncate(truncate_request).await.unwrap(); let state_guard = ingester.state.read().await; - state_guard - .primary_shards - .get(&queue_id_01) - .unwrap() - .assert_publish_position(0); - state_guard - .primary_shards - .get(&queue_id_02) - .unwrap() - .assert_publish_position(1); - - let (position, record) = state_guard - .mrecordlog - .range(&queue_id_01, 0..) - .unwrap() - .next() - .unwrap(); - assert_eq!(position, 1); - assert_eq!(&*record, b"test-doc-001"); + assert_eq!(state_guard.shards.len(), 1); + assert!(state_guard.shards.contains_key(&queue_id_01)); - let record_opt = state_guard + state_guard .mrecordlog - .range(&queue_id_02, 0..) - .unwrap() - .next(); - assert!(record_opt.is_none()); - drop(state_guard); - - // Wait for the removal task to run. - tokio::time::sleep(Duration::from_millis(100)).await; - - let state_guard = ingester.state.read().await; - assert_eq!(state_guard.primary_shards.len(), 1); - assert!(!state_guard.primary_shards.contains_key(&queue_id_02)); - - assert!(!state_guard.mrecordlog.queue_exists(&queue_id_02)); + .assert_records_eq(&queue_id_01, .., &[(1, "\0\0test-doc-011")]); + assert!(!state_guard.shards.contains_key(&queue_id_02)); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs index 561abe84603..f9931e8fe46 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -18,8 +18,6 @@ // along with this program. If not, see . mod fetch; -mod gc; -mod ingest_metastore; mod ingester; mod models; mod mrecord; @@ -31,10 +29,9 @@ mod test_utils; use quickwit_common::tower::Pool; use quickwit_proto::ingest::ingester::IngesterServiceClient; -use quickwit_proto::types::NodeId; +use quickwit_proto::NodeId; pub use self::fetch::MultiFetchStream; -pub use self::ingest_metastore::IngestMetastore; pub use self::ingester::Ingester; pub use self::mrecord::{decoded_mrecords, MRecord}; pub use self::router::IngestRouter; diff --git a/quickwit/quickwit-ingest/src/ingest_v2/models.rs b/quickwit/quickwit-ingest/src/ingest_v2/models.rs index 46583bfdca2..577e28ebade 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/models.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/models.rs @@ -17,262 +17,178 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use std::cmp; +use std::fmt; use quickwit_proto::ingest::ShardState; -use quickwit_proto::NodeId; +use quickwit_proto::types::{NodeId, Position}; use tokio::sync::watch; -#[derive(Debug, Clone, Eq, PartialEq)] -pub(super) struct ShardStatus { - /// Current state of the shard. +/// Shard hosted on a leader node and replicated on a follower node. +pub(super) struct PrimaryShard { + pub follower_id: NodeId, pub shard_state: ShardState, - /// Position up to which indexers have indexed and published the records stored in the shard. - pub publish_position_inclusive: Position, - /// Position up to which the follower has acknowledged replication of the records written in - /// its log. + /// Position of the last record written in the shard's mrecordlog queue. pub replication_position_inclusive: Position, + pub new_records_tx: watch::Sender<()>, + pub new_records_rx: watch::Receiver<()>, } -impl Default for ShardStatus { - fn default() -> Self { +impl fmt::Debug for PrimaryShard { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("PrimaryShard") + .field("follower_id", &self.follower_id) + .field("shard_state", &self.shard_state) + .finish() + } +} + +impl PrimaryShard { + pub fn new(follower_id: NodeId) -> Self { + let (new_records_tx, new_records_rx) = watch::channel(()); Self { + follower_id, shard_state: ShardState::Open, - publish_position_inclusive: Position::default(), - replication_position_inclusive: Position::default(), + replication_position_inclusive: Position::Beginning, + new_records_tx, + new_records_rx, } } } -#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd, Hash)] -pub(super) enum Position { - #[default] - Beginning, - Offset(u64), +/// Shard hosted on a follower node and replicated from a leader node. +pub(super) struct ReplicaShard { + pub leader_id: NodeId, + pub shard_state: ShardState, + /// Position of the last record written in the shard's mrecordlog queue. + pub replication_position_inclusive: Position, + pub new_records_tx: watch::Sender<()>, + pub new_records_rx: watch::Receiver<()>, } -impl Position { - pub fn offset(&self) -> Option { - match self { - Position::Beginning => None, - Position::Offset(offset) => Some(*offset), - } +impl fmt::Debug for ReplicaShard { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("ReplicaShard") + .field("leader_id", &self.leader_id) + .field("shard_state", &self.shard_state) + .finish() } } -impl PartialEq for Position { - fn eq(&self, other: &u64) -> bool { - match self { - Position::Beginning => false, - Position::Offset(offset) => offset == other, +impl ReplicaShard { + pub fn new(leader_id: NodeId) -> Self { + let (new_records_tx, new_records_rx) = watch::channel(()); + Self { + leader_id, + shard_state: ShardState::Open, + replication_position_inclusive: Position::Beginning, + new_records_tx, + new_records_rx, } } } -impl PartialOrd for Position { - fn partial_cmp(&self, other: &u64) -> Option { - match self { - Position::Beginning => Some(cmp::Ordering::Less), - Position::Offset(offset) => offset.partial_cmp(other), - } - } +/// A shard hosted on a single node when the replication factor is set to 1. When a shard is +/// recovered after a node failure, it is always recreated as a solo shard in closed state. +pub(super) struct SoloShard { + pub shard_state: ShardState, + /// Position of the last record written in the shard's mrecordlog queue. + pub replication_position_inclusive: Position, + pub new_records_tx: watch::Sender<()>, + pub new_records_rx: watch::Receiver<()>, } -impl From for Position { - fn from(offset: u64) -> Self { - Position::Offset(offset) +impl fmt::Debug for SoloShard { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + f.debug_struct("SoloShard") + .field("shard_state", &self.shard_state) + .finish() } } -impl From> for Position { - fn from(offset_opt: Option) -> Self { - match offset_opt { - Some(offset) => Position::Offset(offset), - None => Position::Beginning, +impl Default for SoloShard { + fn default() -> Self { + let (new_records_tx, new_records_rx) = watch::channel(()); + Self { + shard_state: ShardState::Open, + replication_position_inclusive: Position::Beginning, + new_records_tx, + new_records_rx, } } } -impl From for Position { - fn from(position_str: String) -> Self { - if position_str.is_empty() { - Position::Beginning - } else { - let offset = position_str - .parse::() - .expect("position should be a u64"); - Position::Offset(offset) +impl SoloShard { + pub fn new(shard_state: ShardState, replication_position_inclusive: Position) -> Self { + Self { + shard_state, + replication_position_inclusive, + ..Default::default() } } } -/// Records the state of a primary shard managed by a leader. -pub(super) struct PrimaryShard { - /// Node ID of the ingester on which the replica shard is hosted. `None` if the replication - /// factor is 1. - pub follower_id_opt: Option, - /// Current state of the shard. - pub shard_state: ShardState, - /// Position up to which indexers have indexed and published the data stored in the shard. - /// It is updated asynchronously in a best effort manner by the indexers and indicates the - /// position up to which the log can be safely truncated. When the shard is closed, the - /// publish position has reached the replication position, and the deletion grace period has - /// passed, the shard can be safely deleted (see [`GcTask`] more details about the deletion - /// logic). - pub publish_position_inclusive: Position, - /// Position up to which the leader has written records in its log. - pub primary_position_inclusive: Position, - /// Position up to which the follower has acknowledged replication of the records written in - /// its log. - pub replica_position_inclusive_opt: Option, - /// Channel to notify readers that new records have been written to the shard. - pub shard_status_tx: watch::Sender, - pub shard_status_rx: watch::Receiver, +pub(super) enum IngesterShard { + /// A primary shard hosted on a leader and replicated on a follower. + Primary(PrimaryShard), + /// A replica shard hosted on a follower. + Replica(ReplicaShard), + /// A shard hosted on a single node when the replication factor is set to 1. + Solo(SoloShard), } -impl PrimaryShard { - pub fn is_removable(&self) -> bool { - self.shard_state.is_closed() - && self.publish_position_inclusive >= self.primary_position_inclusive - } - - pub fn set_publish_position_inclusive( - &mut self, - publish_position_inclusive: impl Into, - ) { - self.publish_position_inclusive = publish_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.publish_position_inclusive = self.publish_position_inclusive; - }); - } - - pub fn set_primary_position_inclusive( - &mut self, - primary_position_inclusive: impl Into, - ) { - self.primary_position_inclusive = primary_position_inclusive.into(); - - // Notify readers if the replication factor is 1. - if self.follower_id_opt.is_none() { - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = self.primary_position_inclusive - }) +impl IngesterShard { + pub fn is_closed(&self) -> bool { + match self { + IngesterShard::Primary(primary_shard) => &primary_shard.shard_state, + IngesterShard::Replica(replica_shard) => &replica_shard.shard_state, + IngesterShard::Solo(solo_shard) => &solo_shard.shard_state, } + .is_closed() } - pub fn set_replica_position_inclusive( - &mut self, - replica_position_inclusive: impl Into, - ) { - assert!(self.follower_id_opt.is_some()); - - let replica_position_inclusive = replica_position_inclusive.into(); - self.replica_position_inclusive_opt = Some(replica_position_inclusive); - - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = replica_position_inclusive - }) - } -} - -/// Records the state of a replica shard managed by a follower. See [`PrimaryShard`] for more -/// details about the fields. -pub(super) struct ReplicaShard { - pub _leader_id: NodeId, - pub shard_state: ShardState, - pub publish_position_inclusive: Position, - pub replica_position_inclusive: Position, - pub shard_status_tx: watch::Sender, - pub shard_status_rx: watch::Receiver, -} - -impl ReplicaShard { - pub fn is_removable(&self) -> bool { - self.shard_state.is_closed() - && self.publish_position_inclusive >= self.replica_position_inclusive - } - - pub fn set_publish_position_inclusive( - &mut self, - publish_position_inclusive: impl Into, - ) { - self.publish_position_inclusive = publish_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.publish_position_inclusive = self.publish_position_inclusive; - }); - } - - pub fn set_replica_position_inclusive( - &mut self, - replica_position_inclusive: impl Into, - ) { - self.replica_position_inclusive = replica_position_inclusive.into(); - self.shard_status_tx.send_modify(|shard_status| { - shard_status.replication_position_inclusive = self.replica_position_inclusive - }); + pub fn replication_position_inclusive(&self) -> Position { + match self { + IngesterShard::Primary(primary_shard) => &primary_shard.replication_position_inclusive, + IngesterShard::Replica(replica_shard) => &replica_shard.replication_position_inclusive, + IngesterShard::Solo(solo_shard) => &solo_shard.replication_position_inclusive, + } + .clone() } -} - -#[cfg(test)] -impl PrimaryShard { - pub(crate) fn for_test( - follower_id_opt: Option<&str>, - shard_state: ShardState, - publish_position_inclusive: impl Into, - primary_position_inclusive: impl Into, - replica_position_inclusive_opt: Option>, - ) -> Self { - let publish_position_inclusive: Position = publish_position_inclusive.into(); - let primary_position_inclusive: Position = primary_position_inclusive.into(); - let replica_position_inclusive_opt: Option = - replica_position_inclusive_opt.map(Into::into); - let replication_position_inclusive = - replica_position_inclusive_opt.unwrap_or(primary_position_inclusive); - let shard_status = ShardStatus { - shard_state, - publish_position_inclusive, - replication_position_inclusive, + pub fn set_replication_position_inclusive(&mut self, replication_position_inclusive: Position) { + if self.replication_position_inclusive() == replication_position_inclusive { + return; + } + match self { + IngesterShard::Primary(primary_shard) => { + primary_shard.replication_position_inclusive = replication_position_inclusive; + } + IngesterShard::Replica(replica_shard) => { + replica_shard.replication_position_inclusive = replication_position_inclusive; + } + IngesterShard::Solo(solo_shard) => { + solo_shard.replication_position_inclusive = replication_position_inclusive; + } }; - let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); + self.notify_new_records(); + } - Self { - follower_id_opt: follower_id_opt.map(Into::into), - shard_state, - publish_position_inclusive, - primary_position_inclusive, - replica_position_inclusive_opt, - shard_status_tx, - shard_status_rx, + pub fn new_records_rx(&self) -> watch::Receiver<()> { + match self { + IngesterShard::Primary(primary_shard) => &primary_shard.new_records_rx, + IngesterShard::Replica(replica_shard) => &replica_shard.new_records_rx, + IngesterShard::Solo(solo_shard) => &solo_shard.new_records_rx, } + .clone() } -} -#[cfg(test)] -impl ReplicaShard { - pub(crate) fn for_test( - leader_id: &str, - shard_state: ShardState, - publish_position_inclusive: impl Into, - replica_position_inclusive: impl Into, - ) -> Self { - let publish_position_inclusive: Position = publish_position_inclusive.into(); - let replica_position_inclusive: Position = replica_position_inclusive.into(); - - let shard_status = ShardStatus { - shard_state, - publish_position_inclusive, - replication_position_inclusive: replica_position_inclusive, - }; - let (shard_status_tx, shard_status_rx) = watch::channel(shard_status); - - Self { - _leader_id: leader_id.into(), - shard_state, - publish_position_inclusive, - replica_position_inclusive, - shard_status_tx, - shard_status_rx, + pub fn notify_new_records(&self) { + match self { + IngesterShard::Primary(primary_shard) => &primary_shard.new_records_tx, + IngesterShard::Replica(replica_shard) => &replica_shard.new_records_tx, + IngesterShard::Solo(solo_shard) => &solo_shard.new_records_tx, } + .send(()) + .expect("channel should be open"); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs b/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs index 9f4822380b0..0d32ba4b52d 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/mrecord.rs @@ -28,24 +28,29 @@ pub enum HeaderVersion { V0 = 0, } -/// MRecord header v0 for a document composed of the header version and the `Doc = 0` record type. -const MRECORD_DOC_HEADER_V0: &[u8; 2] = &[HeaderVersion::V0 as u8, 0]; +/// `Doc` header v0 composed of the header version and the `Doc = 0` record type. +const DOC_HEADER_V0: &[u8; 2] = &[HeaderVersion::V0 as u8, 0]; -/// MRecord header v0 for a commit composed of the header version and the `Commit = 1` record type. -const MRECORD_COMMIT_HEADER_V0: &[u8; 2] = &[HeaderVersion::V0 as u8, 1]; +/// `Commit` header v0 composed of the header version and the `Commit = 1` record type. +const COMMIT_HEADER_V0: &[u8; 2] = &[HeaderVersion::V0 as u8, 1]; + +/// `Eof` header v0 composed of the header version and the `Eof = 2` record type. +const EOF_HEADER_V0: &[u8; 2] = &[HeaderVersion::V0 as u8, 2]; #[derive(Debug, Clone, Eq, PartialEq)] pub enum MRecord { Doc(Bytes), Commit, + Eof, Unknown, } impl MRecord { pub fn encode(&self) -> impl Buf { match &self { - Self::Doc(doc) => MRECORD_DOC_HEADER_V0.chain(doc.clone()), - Self::Commit => MRECORD_COMMIT_HEADER_V0.chain(Bytes::new()), + 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"), } } @@ -62,6 +67,7 @@ impl MRecord { Self::Doc(doc) } 1 => Self::Commit, + 2 => Self::Eof, _ => Self::Unknown, } } @@ -76,6 +82,10 @@ pub fn decoded_mrecords(mrecord_batch: &MRecordBatch) -> impl Iterator bool { + mrecord == EOF_HEADER_V0 +} + #[cfg(test)] mod tests { use super::*; @@ -95,4 +105,18 @@ mod tests { let decoded_record = MRecord::decode(encoded_record); 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/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs index 3f9a482aa8c..4b30096e306 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -28,15 +28,15 @@ use quickwit_proto::ingest::ingester::{ ack_replication_message, syn_replication_message, AckReplicationMessage, ReplicateRequest, ReplicateResponse, ReplicateSuccess, SynReplicationMessage, }; -use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; -use quickwit_proto::types::NodeId; +use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result}; +use quickwit_proto::types::{NodeId, Position}; use tokio::sync::mpsc::error::TryRecvError; -use tokio::sync::{mpsc, oneshot, watch, RwLock}; +use tokio::sync::{mpsc, oneshot, RwLock}; use tokio::task::JoinHandle; use tracing::error; use super::ingester::IngesterState; -use super::models::{Position, ReplicaShard, ShardStatus}; +use super::models::{IngesterShard, ReplicaShard}; use super::mrecord::MRecord; use crate::metrics::INGEST_METRICS; @@ -317,59 +317,38 @@ impl ReplicationTask { for subrequest in replicate_request.subrequests { let queue_id = subrequest.queue_id(); + let from_position_exclusive = subrequest.from_position_exclusive(); + let to_position_inclusive = subrequest.to_position_inclusive(); - let replica_shard: &mut ReplicaShard = if subrequest.from_position_exclusive.is_none() { - // Initialize the replica shard and corresponding mrecordlog queue. - state_guard - .mrecordlog - .create_queue(&queue_id) - .await - .expect("TODO"); - state_guard - .replica_shards - .entry(queue_id.clone()) - .or_insert_with(|| { - let (shard_status_tx, shard_status_rx) = - watch::channel(ShardStatus::default()); - ReplicaShard { - _leader_id: replicate_request.leader_id.clone().into(), - shard_state: ShardState::Open, - publish_position_inclusive: Position::default(), - replica_position_inclusive: Position::default(), - shard_status_tx, - shard_status_rx, - } - }) - } else { - state_guard - .replica_shards - .get_mut(&queue_id) - .expect("replica shard should be initialized") - }; - if replica_shard.shard_state.is_closed() { + let replica_shard: &mut IngesterShard = + if from_position_exclusive == Position::Beginning { + // Initialize the replica shard and corresponding mrecordlog queue. + state_guard + .mrecordlog + .create_queue(&queue_id) + .await + .expect("TODO"); + let leader_id: NodeId = replicate_request.leader_id.clone().into(); + let replica_shard = ReplicaShard::new(leader_id); + let shard = IngesterShard::Replica(replica_shard); + state_guard.shards.entry(queue_id.clone()).or_insert(shard) + } else { + state_guard + .shards + .get_mut(&queue_id) + .expect("replica shard should be initialized") + }; + if replica_shard.is_closed() { // TODO } - let to_position_inclusive = subrequest.to_position_inclusive(); - // let replica_position_inclusive = replica_shard.replica_position_inclusive; - - // TODO: Check if subrequest.from_position_exclusive == replica_position_exclusive. - // If not, check if we should skip the subrequest or not. - // if subrequest.from_position_exclusive != replica_position_exclusive { - // return Err(IngestV2Error::Internal(format!( - // "Bad replica position: expected {}, got {}.", - // subrequest.replica_position_inclusive, replica_position_exclusive - // ))); - let Some(doc_batch) = subrequest.doc_batch else { - let replicate_success = ReplicateSuccess { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - replica_position_inclusive: subrequest.from_position_exclusive, - }; - replicate_successes.push(replicate_success); - continue; - }; - let replica_position_inclusive = if force_commit { + if replica_shard.replication_position_inclusive() != from_position_exclusive { + // TODO + } + let doc_batch = subrequest + .doc_batch + .expect("leader should not send empty replicate subrequests"); + + let current_position_inclusive: Position = if force_commit { let encoded_mrecords = doc_batch .docs() .map(|doc| MRecord::Doc(doc).encode()) @@ -386,7 +365,8 @@ impl ReplicationTask { .append_records(&queue_id, None, encoded_mrecords) .await .expect("TODO") - }; + } + .into(); let batch_num_bytes = doc_batch.num_bytes() as u64; let batch_num_docs = doc_batch.num_docs() as u64; @@ -398,23 +378,23 @@ impl ReplicationTask { .inc_by(batch_num_docs); let replica_shard = state_guard - .replica_shards + .shards .get_mut(&queue_id) .expect("replica shard should exist"); - if replica_position_inclusive != to_position_inclusive { + if current_position_inclusive != to_position_inclusive { return Err(IngestV2Error::Internal(format!( "bad replica position: expected {to_position_inclusive:?}, got \ - {replica_position_inclusive:?}" + {current_position_inclusive:?}" ))); } - replica_shard.set_replica_position_inclusive(replica_position_inclusive); + replica_shard.set_replication_position_inclusive(current_position_inclusive.clone()); let replicate_success = ReplicateSuccess { index_uid: subrequest.index_uid, source_id: subrequest.source_id, shard_id: subrequest.shard_id, - replica_position_inclusive, + replication_position_inclusive: Some(current_position_inclusive), }; replicate_successes.push(replicate_success); } @@ -479,14 +459,13 @@ fn into_replicate_response(ack_replication_message: AckReplicationMessage) -> Re mod tests { use std::collections::HashMap; - use bytes::Bytes; use mrecordlog::MultiRecordLog; use quickwit_proto::ingest::ingester::{ReplicateSubrequest, ReplicateSuccess}; use quickwit_proto::ingest::DocBatchV2; use quickwit_proto::types::queue_id; use super::*; - use crate::ingest_v2::test_utils::{MultiRecordLogTestExt, ReplicaShardTestExt}; + use crate::ingest_v2::test_utils::MultiRecordLogTestExt; #[tokio::test] async fn test_replication_stream_task() { @@ -511,7 +490,7 @@ mod tests { index_uid: subrequest.index_uid.clone(), source_id: subrequest.source_id.clone(), shard_id: subrequest.shard_id, - replica_position_inclusive: subrequest.to_position_inclusive(), + replication_position_inclusive: Some(subrequest.to_position_inclusive()), }) .collect::>(); @@ -539,29 +518,26 @@ mod tests { ReplicateSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, + doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: None, - doc_batch: None, + to_position_inclusive: Some(Position::from(0u64)), }, ReplicateSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: 2, + doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), from_position_exclusive: None, - doc_batch: Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc-010"), - doc_lengths: vec![12], - }), + to_position_inclusive: Some(Position::from(1u64)), }, ReplicateSubrequest { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, - from_position_exclusive: Some(0), - doc_batch: Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc-111test-doc-112"), - doc_lengths: vec![12], - }), + 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)), }, ], replication_seqno: replication_stream_task_handle.next_replication_seqno(), @@ -577,20 +553,20 @@ mod tests { let replicate_success_0 = &replicate_response.successes[0]; assert_eq!(replicate_success_0.index_uid, "test-index:0"); assert_eq!(replicate_success_0.source_id, "test-source"); - assert_eq!(replicate_success_0.shard_id, 0); - assert_eq!(replicate_success_0.replica_position_inclusive, None); - - let replicate_success_0 = &replicate_response.successes[1]; - 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.replica_position_inclusive, Some(0)); + assert_eq!(replicate_success_0.replication_position_inclusive(), 0u64); - let replicate_success_1 = &replicate_response.successes[2]; - assert_eq!(replicate_success_1.index_uid, "test-index:1"); + 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, 1); - assert_eq!(replicate_success_1.replica_position_inclusive, Some(1)); + assert_eq!(replicate_success_1.shard_id, 2); + assert_eq!(replicate_success_1.replication_position_inclusive(), 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); } #[tokio::test] @@ -637,8 +613,7 @@ mod tests { let mrecordlog = MultiRecordLog::open(tempdir.path()).await.unwrap(); let state = Arc::new(RwLock::new(IngesterState { mrecordlog, - primary_shards: HashMap::new(), - replica_shards: HashMap::new(), + shards: HashMap::new(), replication_streams: HashMap::new(), replication_tasks: HashMap::new(), })); @@ -661,29 +636,26 @@ mod tests { ReplicateSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 0, + shard_id: 1, + doc_batch: Some(DocBatchV2::for_test(["test-doc-foo"])), from_position_exclusive: None, - doc_batch: None, + to_position_inclusive: Some(Position::from(0u64)), }, ReplicateSubrequest { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), - shard_id: 1, + shard_id: 2, + doc_batch: Some(DocBatchV2::for_test(["test-doc-bar", "test-doc-baz"])), from_position_exclusive: None, - doc_batch: Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc-010"), - doc_lengths: vec![12], - }), + to_position_inclusive: Some(Position::from(1u64)), }, ReplicateSubrequest { index_uid: "test-index:1".to_string(), source_id: "test-source".to_string(), shard_id: 1, + doc_batch: Some(DocBatchV2::for_test(["test-doc-qux", "test-doc-tux"])), from_position_exclusive: None, - doc_batch: Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc-110test-doc-111"), - doc_lengths: vec![12, 12], - }), + to_position_inclusive: Some(Position::from(1u64)), }, ], replication_seqno: 0, @@ -704,50 +676,43 @@ mod tests { let replicate_success_0 = &replicate_response.successes[0]; assert_eq!(replicate_success_0.index_uid, "test-index:0"); assert_eq!(replicate_success_0.source_id, "test-source"); - assert_eq!(replicate_success_0.shard_id, 0); - assert_eq!(replicate_success_0.replica_position_inclusive, None); + assert_eq!(replicate_success_0.shard_id, 1); + assert_eq!(replicate_success_0.replication_position_inclusive(), 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, 1); - assert_eq!(replicate_success_1.replica_position_inclusive, Some(0)); + assert_eq!(replicate_success_1.shard_id, 2); + assert_eq!(replicate_success_1.replication_position_inclusive(), 1u64); - let replicate_success_1 = &replicate_response.successes[2]; - assert_eq!(replicate_success_1.index_uid, "test-index:1"); - assert_eq!(replicate_success_1.source_id, "test-source"); - assert_eq!(replicate_success_1.shard_id, 1); - assert_eq!(replicate_success_1.replica_position_inclusive, Some(1)); + 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); let state_guard = state.read().await; - assert!(state_guard.primary_shards.is_empty()); - assert_eq!(state_guard.replica_shards.len(), 3); - - let queue_id_00 = queue_id("test-index:0", "test-source", 0); - let replica_shard_00 = state_guard.replica_shards.get(&queue_id_00).unwrap(); - replica_shard_00.assert_is_open(None); + let queue_id_01 = queue_id("test-index:0", "test-source", 1); state_guard .mrecordlog - .assert_records_eq(&queue_id_00, .., &[]); + .assert_records_eq(&queue_id_01, .., &[(0, "\0\0test-doc-foo")]); - let queue_id_01 = queue_id("test-index:0", "test-source", 1); - let replica_shard_01 = state_guard.replica_shards.get(&queue_id_01).unwrap(); - replica_shard_01.assert_is_open(0); + let queue_id_02 = queue_id("test-index:0", "test-source", 2); - state_guard - .mrecordlog - .assert_records_eq(&queue_id_01, .., &[(0, "\0\0test-doc-010")]); + state_guard.mrecordlog.assert_records_eq( + &queue_id_02, + .., + &[(0, "\0\0test-doc-bar"), (1, "\0\0test-doc-baz")], + ); let queue_id_11 = queue_id("test-index:1", "test-source", 1); - let replica_shard_11 = state_guard.replica_shards.get(&queue_id_11).unwrap(); - replica_shard_11.assert_is_open(1); state_guard.mrecordlog.assert_records_eq( &queue_id_11, .., - &[(0, "\0\0test-doc-110"), (1, "\0\0test-doc-111")], + &[(0, "\0\0test-doc-qux"), (1, "\0\0test-doc-tux")], ); drop(state_guard); @@ -759,11 +724,9 @@ mod tests { index_uid: "test-index:0".to_string(), source_id: "test-source".to_string(), shard_id: 1, - from_position_exclusive: Some(0), - doc_batch: Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc-011"), - doc_lengths: vec![12], - }), + doc_batch: Some(DocBatchV2::for_test(["test-doc-moo"])), + from_position_exclusive: Some(Position::from(0u64)), + to_position_inclusive: Some(Position::from(1u64)), }], replication_seqno: 1, }; @@ -784,16 +747,14 @@ 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.replica_position_inclusive, Some(1)); + assert_eq!(replicate_success_0.replication_position_inclusive(), 1u64); let state_guard = state.read().await; state_guard.mrecordlog.assert_records_eq( &queue_id_01, .., - &[(0, "\0\0test-doc-010"), (1, "\0\0test-doc-011")], + &[(0, "\0\0test-doc-foo"), (1, "\0\0test-doc-moo")], ); - let replica_shard_01 = state_guard.replica_shards.get(&queue_id_01).unwrap(); - replica_shard_01.assert_is_open(1); } } diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index 24a0af547dc..8528ebc71fe 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -33,8 +33,7 @@ use quickwit_proto::ingest::router::{ IngestRequestV2, IngestResponseV2, IngestRouterService, IngestSubrequest, }; use quickwit_proto::ingest::IngestV2Result; -use quickwit_proto::types::NodeId; -use quickwit_proto::IndexUid; +use quickwit_proto::types::{IndexUid, NodeId}; use tokio::sync::RwLock; use super::shard_table::ShardTable; @@ -167,7 +166,7 @@ impl IngestRouterService for IngestRouter { for ingest_subrequest in ingest_request.subrequests { let shard = state_guard .shard_table - .find_entry(&*ingest_subrequest.index_id, &ingest_subrequest.source_id) + .find_entry(&ingest_subrequest.index_id, &ingest_subrequest.source_id) .expect("TODO") .next_shard_round_robin(); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs index a1089155b1f..88e24f9f57e 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs @@ -21,8 +21,7 @@ use std::collections::HashMap; use std::sync::atomic::{AtomicUsize, Ordering}; use quickwit_proto::ingest::Shard; -use quickwit_proto::types::SourceId; -use quickwit_proto::{IndexId, NodeId}; +use quickwit_proto::types::{IndexId, NodeId, SourceId}; /// A set of open shards for a given index and source. #[derive(Debug, Default)] diff --git a/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs index 07bc93ee0ea..6954a71cfac 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs @@ -20,9 +20,63 @@ use std::ops::RangeBounds; use mrecordlog::MultiRecordLog; -use quickwit_proto::ingest::ShardState; +use quickwit_proto::types::Position; -use super::models::{Position, PrimaryShard, ReplicaShard}; +use super::models::IngesterShard; + +pub(super) trait IngesterShardTestExt { + fn assert_is_solo(&self); + + fn assert_is_primary(&self); + + fn assert_is_replica(&self); + + fn assert_is_open(&self); + + fn assert_is_closed(&self); + + fn assert_replication_position(&self, expected_replication_position: impl Into); +} + +impl IngesterShardTestExt for IngesterShard { + #[track_caller] + fn assert_is_solo(&self) { + assert!(matches!(self, IngesterShard::Solo(_))) + } + + #[track_caller] + fn assert_is_primary(&self) { + assert!(matches!(self, IngesterShard::Primary(_))) + } + + #[track_caller] + fn assert_is_replica(&self) { + assert!(matches!(self, IngesterShard::Replica(_))) + } + + #[track_caller] + fn assert_is_open(&self) { + assert!(!self.is_closed()) + } + + #[track_caller] + fn assert_is_closed(&self) { + assert!(self.is_closed()) + } + + #[track_caller] + fn assert_replication_position(&self, expected_replication_position: impl Into) { + let expected_replication_position = expected_replication_position.into(); + + assert_eq!( + self.replication_position_inclusive(), + expected_replication_position, + "expected replication position at `{:?}`, got `{:?}`", + expected_replication_position, + self.replication_position_inclusive() + ); + } +} pub(super) trait MultiRecordLogTestExt { fn assert_records_eq(&self, queue_id: &str, range: R, expected_records: &[(u64, &str)]) @@ -59,107 +113,3 @@ impl MultiRecordLogTestExt for MultiRecordLog { } } } - -pub(super) trait PrimaryShardTestExt { - fn assert_positions( - &self, - expected_primary_position: impl Into, - expected_replica_position: Option>, - ); - - fn assert_publish_position(&self, expected_publish_position: impl Into); - - fn assert_is_open(&self, expected_position: impl Into); -} - -impl PrimaryShardTestExt for PrimaryShard { - #[track_caller] - fn assert_positions( - &self, - expected_primary_position: impl Into, - expected_replica_position: Option>, - ) { - let expected_primary_position = expected_primary_position.into(); - let expected_replica_position = - expected_replica_position.map(|replication_position| replication_position.into()); - - assert_eq!( - self.primary_position_inclusive, expected_primary_position, - "expected primary position at `{:?}`, got `{:?}`", - expected_primary_position, self.primary_position_inclusive - ); - assert_eq!( - self.replica_position_inclusive_opt, expected_replica_position, - "expected replica position at `{:?}`, got `{:?}`", - expected_replica_position, self.replica_position_inclusive_opt - ); - } - - #[track_caller] - fn assert_publish_position(&self, expected_publish_position: impl Into) { - let expected_publish_position = expected_publish_position.into(); - - assert_eq!( - self.publish_position_inclusive, expected_publish_position, - "expected publish position at `{:?}`, got `{:?}`", - expected_publish_position, self.publish_position_inclusive - ); - assert_eq!( - self.shard_status_tx.borrow().publish_position_inclusive, - expected_publish_position, - "expected publish position at `{:?}`, got `{:?}`", - expected_publish_position, - self.publish_position_inclusive - ); - } - - #[track_caller] - fn assert_is_open(&self, expected_replication_position: impl Into) { - let expected_replication_position = expected_replication_position.into(); - let shard_status = self.shard_status_tx.borrow(); - assert_eq!( - shard_status.shard_state, - ShardState::Open, - "expected open primary shard, got closed one", - ); - assert_eq!( - shard_status.replication_position_inclusive, expected_replication_position, - "expected open primary shard at `{expected_replication_position:?}`, got `{:?}`", - shard_status.replication_position_inclusive - ); - } -} - -pub(super) trait ReplicaShardTestExt { - fn assert_position(&self, expected_replica_position: impl Into); - - fn assert_is_open(&self, expected_position: impl Into); -} - -impl ReplicaShardTestExt for ReplicaShard { - #[track_caller] - fn assert_position(&self, expected_replica_position: impl Into) { - let expected_replica_position = expected_replica_position.into(); - assert_eq!( - self.replica_position_inclusive, expected_replica_position, - "expected replica position at `{:?}`, got `{:?}`", - expected_replica_position, self.replica_position_inclusive - ); - } - - #[track_caller] - fn assert_is_open(&self, expected_replication_position: impl Into) { - let expected_replication_position = expected_replication_position.into(); - let shard_status = self.shard_status_tx.borrow(); - assert_eq!( - shard_status.shard_state, - ShardState::Open, - "expected open replica shard, got closed one", - ); - assert_eq!( - shard_status.replication_position_inclusive, expected_replication_position, - "expected open replica shard at `{expected_replication_position:?}`, got `{:?}`", - shard_status.replication_position_inclusive - ); - } -} diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 19e5adc5d71..ae0cfec5e7f 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -157,9 +157,7 @@ impl DeleteTaskPipeline { ); let index_config = self .metastore - .index_metadata(IndexMetadataRequest::for_index_uid( - self.index_uid.to_string(), - )) + .index_metadata(IndexMetadataRequest::for_index_uid(self.index_uid.clone())) .await? .deserialize_index_metadata()? .into_index_config(); diff --git a/quickwit/quickwit-metastore/src/checkpoint.rs b/quickwit/quickwit-metastore/src/checkpoint.rs index e3a50fc4296..2ccaee9f710 100644 --- a/quickwit/quickwit-metastore/src/checkpoint.rs +++ b/quickwit/quickwit-metastore/src/checkpoint.rs @@ -25,7 +25,7 @@ use std::iter::FromIterator; use std::ops::Range; use std::sync::Arc; -use quickwit_proto::SourceId; +use quickwit_proto::types::{Position, SourceId}; use serde::ser::SerializeMap; use serde::{Deserialize, Serialize}; use thiserror::Error; @@ -79,85 +79,6 @@ impl From for PartitionId { } } -/// Marks a position within a specific partition of a source. -/// -/// The nature of the position may very depending on the source. -/// Each source needs to encode it as a `String` in such a way that -/// the lexicographical order matches the natural order of the -/// position. -/// -/// For instance, for u64, a 20-left-padded decimal representation -/// can be used. Alternatively, a base64 representation of their -/// Big Endian representation can be used. -/// -/// The empty string can be used to represent the beginning of the source, -/// if no position makes sense. It can be built via `Position::default()`. -#[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Ord, PartialOrd, Serialize, Deserialize)] -pub enum Position { - #[default] - Beginning, - Offset(Arc), -} - -impl Position { - /// String representation of the position. - pub fn as_str(&self) -> &str { - match self { - Position::Beginning => "", - Position::Offset(offset) => offset, - } - } - - /// Returns the position as a `i64` (Kafka source). - pub fn as_i64(&self) -> Option { - match self { - Position::Beginning => None, - Position::Offset(offset) => offset.parse::().ok(), - } - } - - /// Returns the position as a `u64` (ingest). - pub fn as_u64(&self) -> Option { - match self { - Position::Beginning => None, - Position::Offset(offset) => offset.parse::().ok(), - } - } -} - -impl From for Position { - fn from(offset: i64) -> Self { - assert!(offset >= 0); - let offset_str = format!("{offset:0>20}"); - Position::Offset(Arc::new(offset_str)) - } -} - -impl From for Position { - fn from(offset: u64) -> Self { - let offset_str = format!("{offset:0>20}"); - Position::Offset(Arc::new(offset_str)) - } -} - -impl From for Position { - fn from(position_str: String) -> Self { - match position_str.as_str() { - "" => Position::Beginning, - _ => Position::Offset(Arc::new(position_str)), - } - } -} - -impl From<&str> for Position { - fn from(position_str: &str) -> Self { - match position_str { - "" => Position::Beginning, - _ => Position::Offset(Arc::new(position_str.to_string())), - } - } -} - /// A partition delta represents an interval (from, to] over a partition of a source. #[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] pub struct PartitionDelta { @@ -269,7 +190,9 @@ impl SourceCheckpoint { /// Creates a checkpoint from an iterator of `(PartitionId, Position)` tuples. /// ``` -/// use quickwit_metastore::checkpoint::{SourceCheckpoint, PartitionId, Position}; +/// use quickwit_metastore::checkpoint::{SourceCheckpoint, PartitionId}; +/// use quickwit_proto::types::Position; +/// /// let checkpoint: SourceCheckpoint = [(0u64, 0u64), (1u64, 2u64)] /// .into_iter() /// .map(|(partition_id, offset)| { diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 762dc14838c..13778862823 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -23,17 +23,16 @@ use async_trait::async_trait; use quickwit_common::uri::Uri; use quickwit_proto::control_plane::{ControlPlaneService, ControlPlaneServiceClient}; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CloseShardsRequest, - CloseShardsResponse, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, - DeleteShardsRequest, DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, - DeleteTask, EmptyResponse, IndexMetadataRequest, IndexMetadataResponse, - LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, - ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, - MetastoreServiceClient, OpenShardsRequest, OpenShardsResponse, PublishSplitsRequest, - ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, + CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, + IndexMetadataRequest, IndexMetadataResponse, LastDeleteOpstampRequest, + LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, + ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, + ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, + MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardsRequest, + OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, + ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; /// A [`MetastoreService`] implementation that proxies some requests to the control plane so it can @@ -230,13 +229,6 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.list_shards(request).await } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> MetastoreResult { - self.metastore.close_shards(request).await - } - async fn delete_shards( &mut self, request: DeleteShardsRequest, diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs index 0c7a23718db..5514807ed22 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/mod.rs @@ -28,16 +28,15 @@ use std::collections::HashMap; use std::fmt::Debug; use std::ops::Bound; -use itertools::{Either, Itertools}; +use itertools::Itertools; use quickwit_common::PrettySample; use quickwit_config::{SourceConfig, INGEST_SOURCE_ID}; use quickwit_proto::metastore::{ - AcquireShardsSubrequest, AcquireShardsSubresponse, CloseShardsFailure, CloseShardsSubrequest, - CloseShardsSuccess, DeleteQuery, DeleteShardsSubrequest, DeleteTask, EntityKind, - ListShardsSubrequest, ListShardsSubresponse, MetastoreError, MetastoreResult, - OpenShardsSubrequest, OpenShardsSubresponse, + AcquireShardsSubrequest, AcquireShardsSubresponse, DeleteQuery, DeleteShardsSubrequest, + DeleteTask, EntityKind, ListShardsSubrequest, ListShardsSubresponse, MetastoreError, + MetastoreResult, OpenShardsSubrequest, OpenShardsSubresponse, }; -use quickwit_proto::{IndexUid, PublishToken, SourceId, SplitId}; +use quickwit_proto::types::{IndexUid, PublishToken, SourceId, SplitId}; use serde::{Deserialize, Serialize}; use serialize::VersionedFileBackedIndex; use shards::Shards; @@ -623,34 +622,6 @@ impl FileBackedIndex { } } - pub(crate) fn close_shards( - &mut self, - subrequests: Vec, - ) -> MetastoreResult>>> - { - let mut mutation_occurred = false; - let mut subresponses = Vec::with_capacity(subrequests.len()); - - for subrequest in subrequests { - let subresponse = match self - .get_shards_for_source_mut(&subrequest.source_id)? - .close_shards(subrequest)? - { - MutationOccurred::Yes(subresponse) => { - mutation_occurred = true; - subresponse - } - MutationOccurred::No(subresponse) => subresponse, - }; - subresponses.push(subresponse); - } - if mutation_occurred { - Ok(MutationOccurred::Yes(subresponses)) - } else { - Ok(MutationOccurred::No(subresponses)) - } - } - pub(crate) fn delete_shards( &mut self, subrequests: Vec, 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 1440471df9a..8463ca416ad 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 @@ -21,20 +21,17 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use std::fmt; -use itertools::Either; use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ - AcquireShardsSubrequest, AcquireShardsSubresponse, CloseShardsFailure, CloseShardsFailureKind, - CloseShardsSubrequest, CloseShardsSuccess, DeleteShardsSubrequest, EntityKind, + AcquireShardsSubrequest, AcquireShardsSubresponse, DeleteShardsSubrequest, EntityKind, ListShardsSubrequest, ListShardsSubresponse, MetastoreError, MetastoreResult, OpenShardsSubrequest, OpenShardsSubresponse, }; -use quickwit_proto::types::ShardId; -use quickwit_proto::{queue_id, IndexUid, SourceId}; +use quickwit_proto::types::{queue_id, IndexUid, Position, ShardId, SourceId}; use serde::{Deserialize, Serialize}; use tracing::{info, warn}; -use crate::checkpoint::{PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta}; +use crate::checkpoint::{PartitionId, SourceCheckpoint, SourceCheckpointDelta}; use crate::file_backed_metastore::MutationOccurred; /// Manages the shards of a source. @@ -79,7 +76,7 @@ impl Shards { for shard in serde_shards.shards { checkpoint.add_partition( PartitionId::from(shard.shard_id), - Position::from(shard.publish_position_inclusive.clone()), + shard.publish_position_inclusive(), ); shards.insert(shard.shard_id, shard); } @@ -201,84 +198,23 @@ impl Shards { } } - pub(super) fn close_shards( - &mut self, - subrequest: CloseShardsSubrequest, - ) -> MetastoreResult>> { - let Some(shard) = self.shards.get_mut(&subrequest.shard_id) else { - let failure = CloseShardsFailure { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - failure_kind: CloseShardsFailureKind::NotFound as i32, - failure_message: "shard not found".to_string(), - }; - return Ok(MutationOccurred::No(Either::Right(failure))); - }; - match subrequest.shard_state() { - ShardState::Closing => { - shard.shard_state = ShardState::Closing as i32; - info!( - index_id=%self.index_uid.index_id(), - source_id=%shard.source_id, - shard_id=%shard.shard_id, - "Closing shard.", - ); - } - ShardState::Closed => { - shard.shard_state = ShardState::Closed as i32; - shard.replication_position_inclusive = shard - .replication_position_inclusive - .min(subrequest.replication_position_inclusive); - info!( - index_id=%self.index_uid.index_id(), - source_id=%shard.source_id, - shard_id=%shard.shard_id, - "closed shard", - ); - } - other => { - let failure_message = format!( - "invalid `shard_state` argument: expected `Closing` or `Closed` state, got \ - `{other:?}`.", - ); - let failure = CloseShardsFailure { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - failure_kind: CloseShardsFailureKind::InvalidArgument as i32, - failure_message, - }; - return Ok(MutationOccurred::No(Either::Right(failure))); - } - } - let success = CloseShardsSuccess { - index_uid: subrequest.index_uid, - source_id: subrequest.source_id, - shard_id: subrequest.shard_id, - leader_id: shard.leader_id.clone(), - follower_id: shard.follower_id.clone(), - publish_position_inclusive: shard.publish_position_inclusive.clone(), - }; - Ok(MutationOccurred::Yes(Either::Left(success))) - } - pub(super) fn delete_shards( &mut self, subrequest: DeleteShardsSubrequest, force: bool, ) -> MetastoreResult> { let mut mutation_occurred = false; + for shard_id in subrequest.shard_ids { if let Entry::Occupied(entry) = self.shards.entry(shard_id) { let shard = entry.get(); - if force || shard.is_deletable() { + if force || shard.publish_position_inclusive() == Position::Eof { mutation_occurred = true; info!( index_id=%self.index_uid.index_id(), source_id=%self.source_id, shard_id=%shard.shard_id, - "Deleted shard.", + "deleted shard", ); entry.remove(); continue; @@ -332,7 +268,7 @@ impl Shards { let message = "failed to apply checkpoint delta: invalid publish token".to_string(); return Err(MetastoreError::InvalidArgument { message }); } - let publish_position_inclusive = partition_delta.to.as_str().to_string(); + let publish_position_inclusive = partition_delta.to; shard_ids.push((shard_id, publish_position_inclusive)) } self.checkpoint @@ -341,7 +277,11 @@ impl Shards { for (shard_id, publish_position_inclusive) in shard_ids { let shard = self.get_shard_mut(shard_id).expect("shard should exist"); - shard.publish_position_inclusive = publish_position_inclusive + + if publish_position_inclusive == Position::Eof { + shard.shard_state = ShardState::Closed as i32; + } + shard.publish_position_inclusive = Some(publish_position_inclusive); } Ok(MutationOccurred::Yes(())) } @@ -377,6 +317,8 @@ impl From for SerdeShards { #[cfg(test)] mod tests { + use quickwit_proto::ingest::ShardState; + use super::*; #[test] @@ -407,8 +349,7 @@ mod tests { assert_eq!(shard.shard_state, 0); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id, None); - assert_eq!(shard.replication_position_inclusive, None); - assert_eq!(shard.publish_position_inclusive, ""); + assert_eq!(shard.publish_position_inclusive(), Position::Beginning); assert_eq!(shards.shards.get(&1).unwrap(), shard); @@ -441,8 +382,7 @@ mod tests { assert_eq!(shard.shard_state, 0); assert_eq!(shard.leader_id, "leader_id"); assert_eq!(shard.follower_id.as_ref().unwrap(), "follower_id"); - assert_eq!(shard.replication_position_inclusive, None); - assert_eq!(shard.publish_position_inclusive, ""); + assert_eq!(shard.publish_position_inclusive(), Position::Beginning); assert_eq!(shards.shards.get(&2).unwrap(), shard); } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs index 4d926c99f5a..baa5460db96 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/mod.rs @@ -33,12 +33,11 @@ use std::time::Duration; use async_trait::async_trait; use futures::future::try_join_all; -use itertools::{Either, Itertools}; +use itertools::Itertools; use quickwit_config::validate_index_id_pattern; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, AcquireShardsSubrequest, AddSourceRequest, - CloseShardsRequest, CloseShardsResponse, CloseShardsSubrequest, CreateIndexRequest, - CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, + CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteShardsSubrequest, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, EntityKind, IndexMetadataRequest, IndexMetadataResponse, LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, @@ -758,35 +757,6 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> MetastoreResult { - let mut response = CloseShardsResponse { - successes: Vec::with_capacity(request.subrequests.len()), - failures: Vec::new(), - }; - // We must group the subrequests by `index_uid` to mutate each index only once, since each - // mutation triggers an IO. - let grouped_subrequests: HashMap> = request - .subrequests - .into_iter() - .into_group_map_by(|subrequest| IndexUid::new(subrequest.index_uid.clone())); - - for (index_uid, subrequests) in grouped_subrequests { - let subresponses = self - .mutate(index_uid, |index| index.close_shards(subrequests)) - .await?; - for subresponse in subresponses { - match subresponse { - Either::Left(success) => response.successes.push(success), - Either::Right(failure) => response.failures.push(failure), - } - } - } - Ok(response) - } - async fn delete_shards( &mut self, request: DeleteShardsRequest, diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index e8144d3d163..050a9317893 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -25,15 +25,13 @@ use std::collections::{BTreeMap, HashMap}; use quickwit_common::uri::Uri; use quickwit_config::{IndexConfig, SourceConfig, TestableForRegression}; use quickwit_proto::metastore::{EntityKind, MetastoreError, MetastoreResult}; -use quickwit_proto::{IndexUid, SourceId}; +use quickwit_proto::types::{IndexUid, Position, SourceId}; use serde::{Deserialize, Serialize}; use serialize::VersionedIndexMetadata; use time::OffsetDateTime; use ulid::Ulid; -use crate::checkpoint::{ - IndexCheckpoint, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, -}; +use crate::checkpoint::{IndexCheckpoint, PartitionId, SourceCheckpoint, SourceCheckpointDelta}; /// An index metadata carries all meta data about an index. #[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] diff --git a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs index b273de37c2e..bff32567eef 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgresql_metastore.rs @@ -33,17 +33,16 @@ use quickwit_config::{ }; use quickwit_doc_mapper::tag_pruning::TagFilterAst; use quickwit_proto::metastore::{ - AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CloseShardsRequest, - CloseShardsResponse, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteQuery, - DeleteShardsRequest, DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, - DeleteTask, EmptyResponse, EntityKind, IndexMetadataRequest, IndexMetadataResponse, - LastDeleteOpstampRequest, LastDeleteOpstampResponse, ListDeleteTasksRequest, - ListDeleteTasksResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, - ListShardsRequest, ListShardsResponse, ListSplitsRequest, ListSplitsResponse, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, - MetastoreService, MetastoreServiceClient, OpenShardsRequest, OpenShardsResponse, - PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + AcquireShardsRequest, AcquireShardsResponse, AddSourceRequest, CreateIndexRequest, + CreateIndexResponse, DeleteIndexRequest, DeleteQuery, DeleteShardsRequest, + DeleteShardsResponse, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, EmptyResponse, + EntityKind, IndexMetadataRequest, IndexMetadataResponse, LastDeleteOpstampRequest, + LastDeleteOpstampResponse, ListDeleteTasksRequest, ListDeleteTasksResponse, + ListIndexesMetadataRequest, ListIndexesMetadataResponse, ListShardsRequest, ListShardsResponse, + ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, + MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, OpenShardsRequest, + OpenShardsResponse, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, + ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::IndexUid; use sqlx::migrate::Migrator; @@ -1311,13 +1310,6 @@ impl MetastoreService for PostgresqlMetastore { unimplemented!("`close_shards` is not implemented for PostgreSQL metastore") } - async fn close_shards( - &mut self, - _request: CloseShardsRequest, - ) -> MetastoreResult { - unimplemented!("`close_shards` is not implemented for PostgreSQL metastore") - } - async fn list_shards( &mut self, _request: ListShardsRequest, diff --git a/quickwit/quickwit-metastore/src/tests.rs b/quickwit/quickwit-metastore/src/tests.rs index 8d9d2c8b68c..57acd026610 100644 --- a/quickwit/quickwit-metastore/src/tests.rs +++ b/quickwit/quickwit-metastore/src/tests.rs @@ -37,14 +37,14 @@ pub mod test_suite { PublishSplitsRequest, ResetSourceCheckpointRequest, SourceType, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, }; - use quickwit_proto::IndexUid; + use quickwit_proto::{IndexUid, Position}; use quickwit_query::query_ast::qast_json_helper; use time::OffsetDateTime; use tokio::time::sleep; use tracing::{error, info}; use crate::checkpoint::{ - IndexCheckpointDelta, PartitionId, Position, SourceCheckpoint, SourceCheckpointDelta, + IndexCheckpointDelta, PartitionId, SourceCheckpoint, SourceCheckpointDelta, }; use crate::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, diff --git a/quickwit/quickwit-proto/Cargo.toml b/quickwit/quickwit-proto/Cargo.toml index dec5bd97a47..4e118dcf3ef 100644 --- a/quickwit/quickwit-proto/Cargo.toml +++ b/quickwit/quickwit-proto/Cargo.toml @@ -13,6 +13,7 @@ documentation = "https://quickwit.io/docs/" anyhow = { workspace = true } async-trait = { workspace = true } bytes = { workspace = true } +bytestring = { workspace = true } dyn-clone = { workspace = true } http = { workspace = true } hyper = { workspace = true } diff --git a/quickwit/quickwit-proto/build.rs b/quickwit/quickwit-proto/build.rs index 6ed886a0534..e955d37d5d5 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -79,7 +79,12 @@ fn main() -> Result<(), Box> { // Ingest service (metastore service proto should be generated before ingest). let mut prost_config = prost_build::Config::default(); prost_config - .bytes(["DocBatchV2.doc_buffer", "MRecordBatch.mrecord_buffer"]) + .bytes([ + "DocBatchV2.doc_buffer", + "MRecordBatch.mrecord_buffer", + "Position.position", + ]) + .extern_path(".quickwit.ingest.Position", "crate::types::Position") .type_attribute("Shard", "#[derive(Eq)]") .field_attribute( "Shard.follower_id", @@ -87,7 +92,7 @@ fn main() -> Result<(), Box> { ) .field_attribute( "Shard.publish_position_inclusive", - "#[serde(default, skip_serializing_if = \"String::is_empty\")]", + "#[serde(default, skip_serializing_if = \"Option::is_none\")]", ) .field_attribute( "Shard.publish_token", diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 82276520d12..25a877c06cb 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -59,16 +59,6 @@ service ControlPlaneService { // Returns the list of open shards for one or several sources. If the control plane is not able to find any // for a source, it will pick a pair of leader-follower ingesters and will open a new shard. rpc GetOrCreateOpenShards(GetOrCreateOpenShardsRequest) returns (GetOrCreateOpenShardsResponse); - - // The following RPCs are are metastore callbacks: - // - `close_shards` - // - `delete_shards` - - // Closes a list of shards. This RPC is a metastore callback. - rpc CloseShards(quickwit.metastore.CloseShardsRequest) returns (quickwit.metastore.EmptyResponse); - - // Deletes a list of shards. This RPC is a metastore callback. - rpc DeleteShards(quickwit.metastore.DeleteShardsRequest) returns (quickwit.metastore.EmptyResponse); } // Shard API diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto index 655fd034037..9208f93a7ec 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingest.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -21,6 +21,13 @@ syntax = "proto3"; package quickwit.ingest; +// The corresponding Rust struct [`crate::types::Position`] is defined manually and +// externally provided during code generation (see `build.rs`). +// +// Modify at your own risk. +message Position { + bytes position = 1; +} enum CommitTypeV2 { AUTO = 0; @@ -43,9 +50,8 @@ message MRecordBatch { enum ShardState { // The shard is open and accepts write requests. OPEN = 0; - // The shard is open and still accepts write requests, but should no longer be advertised to ingest routers. - // It is waiting for its leader or follower to close it with its final replication position, after which write requests will be rejected. - CLOSING = 1; + // The ingester hosting the shard is unavailable. + UNAVAILABLE = 1; // The shard is closed and cannot be written to. // It can be safely deleted if the publish position is superior or equal to the replication position. CLOSED = 2; @@ -63,12 +69,10 @@ message Shard { // Mutable fields ShardState shard_state = 8; - // Position up to which the follower has acknowledged replication of the records written in its log. - optional uint64 replication_position_inclusive = 9; // Position up to which indexers have indexed and published the records stored in the shard. // It is updated asynchronously in a best effort manner by the indexers and indicates the position up to which the log can be safely truncated. - string publish_position_inclusive = 10; + Position publish_position_inclusive = 9; // A publish token that ensures only one indexer works on a given shard at a time. // For instance, if an indexer goes rogue, eventually the control plane will detect it and assign the shard to another indexer, which will override the publish token. - optional string publish_token = 11; + optional string publish_token = 10; } diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto index c5156efa720..367fff025b4 100644 --- a/quickwit/quickwit-proto/protos/quickwit/ingester.proto +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -23,7 +23,6 @@ package quickwit.ingest.ingester; import "quickwit/ingest.proto"; - service IngesterService { // Persists batches of documents to primary shards owned by a leader. rpc Persist(PersistRequest) returns (PersistResponse); @@ -67,7 +66,7 @@ message PersistSuccess { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - optional uint64 replication_position_inclusive = 4; + quickwit.ingest.Position replication_position_inclusive = 4; } enum PersistFailureKind { @@ -116,7 +115,8 @@ message ReplicateSubrequest { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - optional uint64 from_position_exclusive = 4; + quickwit.ingest.Position from_position_exclusive = 4; + quickwit.ingest.Position to_position_inclusive = 5; ingest.DocBatchV2 doc_batch = 6; } @@ -132,7 +132,7 @@ message ReplicateSuccess { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - optional uint64 replica_position_inclusive = 4; + quickwit.ingest.Position replication_position_inclusive = 4; } message ReplicateFailure { @@ -152,7 +152,7 @@ message TruncateSubrequest { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - uint64 to_position_inclusive = 4; + quickwit.ingest.Position to_position_inclusive = 4; } message TruncateResponse { @@ -164,16 +164,17 @@ message OpenFetchStreamRequest { string index_uid = 2; string source_id = 3; uint64 shard_id = 4; - optional uint64 from_position_exclusive = 5; - optional uint64 to_position_inclusive = 6; + quickwit.ingest.Position from_position_exclusive = 5; + quickwit.ingest.Position to_position_inclusive = 6; } message FetchResponseV2 { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - uint64 from_position_inclusive = 4; - quickwit.ingest.MRecordBatch mrecord_batch = 5; + quickwit.ingest.MRecordBatch mrecord_batch = 4; + quickwit.ingest.Position from_position_exclusive = 5; + quickwit.ingest.Position to_position_inclusive = 6; } message PingRequest { diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 5824faaa9dd..89fe91146bf 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -108,8 +108,6 @@ service MetastoreService { // list of acquired shards along with the positions to index from. rpc AcquireShards(AcquireShardsRequest) returns (AcquireShardsResponse); - rpc CloseShards(CloseShardsRequest) returns (CloseShardsResponse); - rpc DeleteShards(DeleteShardsRequest) returns (DeleteShardsResponse); rpc ListShards(ListShardsRequest) returns (ListShardsResponse); @@ -303,45 +301,6 @@ message AcquireShardsSubresponse { repeated quickwit.ingest.Shard acquired_shards = 3; } -message CloseShardsRequest { - repeated CloseShardsSubrequest subrequests = 1; -} - -message CloseShardsSubrequest { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - quickwit.ingest.ShardState shard_state = 4; - optional uint64 replication_position_inclusive = 5; -} - -message CloseShardsResponse { - repeated CloseShardsSuccess successes = 1; - repeated CloseShardsFailure failures = 2; -} - -message CloseShardsSuccess { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - string leader_id = 4; - optional string follower_id = 5; - string publish_position_inclusive = 6; -} - -enum CloseShardsFailureKind { - INVALID_ARGUMENT = 0; - NOT_FOUND = 1; -} - -message CloseShardsFailure { - string index_uid = 1; - string source_id = 2; - uint64 shard_id = 3; - CloseShardsFailureKind failure_kind = 4; - string failure_message = 5; -} - message DeleteShardsRequest { repeated DeleteShardsSubrequest subrequests = 1; bool force = 2; diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto index b3bfd26071a..1921ebad44e 100644 --- a/quickwit/quickwit-proto/protos/quickwit/router.proto +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -50,7 +50,8 @@ message IngestSuccess { string index_uid = 1; string source_id = 2; uint64 shard_id = 3; - uint64 replication_position = 4; + // Replication position inclusive. + quickwit.ingest.Position replication_position_inclusive = 4; } message IngestFailure { diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index c4527d7f865..3cf10b798ca 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -73,16 +73,6 @@ pub trait ControlPlaneService: std::fmt::Debug + dyn_clone::DynClone + Send + Sy &mut self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult; - /// Closes a list of shards. This RPC is a metastore callback. - async fn close_shards( - &mut self, - request: super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult; - /// Deletes a list of shards. This RPC is a metastore callback. - async fn delete_shards( - &mut self, - request: super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult; } dyn_clone::clone_trait_object!(ControlPlaneService); #[cfg(any(test, feature = "testsuite"))] @@ -198,18 +188,6 @@ impl ControlPlaneService for ControlPlaneServiceClient { ) -> crate::control_plane::ControlPlaneResult { self.inner.get_or_create_open_shards(request).await } - async fn close_shards( - &mut self, - request: super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner.close_shards(request).await - } - async fn delete_shards( - &mut self, - request: super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner.delete_shards(request).await - } } #[cfg(any(test, feature = "testsuite"))] pub mod control_plane_service_mock { @@ -268,22 +246,6 @@ pub mod control_plane_service_mock { > { self.inner.lock().await.get_or_create_open_shards(request).await } - async fn close_shards( - &mut self, - request: super::super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult< - super::super::metastore::EmptyResponse, - > { - self.inner.lock().await.close_shards(request).await - } - async fn delete_shards( - &mut self, - request: super::super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult< - super::super::metastore::EmptyResponse, - > { - self.inner.lock().await.delete_shards(request).await - } } impl From for ControlPlaneServiceClient { fn from(mock: MockControlPlaneService) -> Self { @@ -398,40 +360,6 @@ impl tower::Service for Box -for Box { - type Response = super::metastore::EmptyResponse; - type Error = crate::control_plane::ControlPlaneError; - type Future = BoxFuture; - fn poll_ready( - &mut self, - _cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - std::task::Poll::Ready(Ok(())) - } - fn call(&mut self, request: super::metastore::CloseShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.close_shards(request).await }; - Box::pin(fut) - } -} -impl tower::Service -for Box { - type Response = super::metastore::EmptyResponse; - type Error = crate::control_plane::ControlPlaneError; - type Future = BoxFuture; - fn poll_ready( - &mut self, - _cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - std::task::Poll::Ready(Ok(())) - } - fn call(&mut self, request: super::metastore::DeleteShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_shards(request).await }; - Box::pin(fut) - } -} /// A tower block is a set of towers. Each tower is stack of layers (middlewares) that are applied to a service. #[derive(Debug)] struct ControlPlaneServiceTowerBlock { @@ -466,16 +394,6 @@ struct ControlPlaneServiceTowerBlock { GetOrCreateOpenShardsResponse, crate::control_plane::ControlPlaneError, >, - close_shards_svc: quickwit_common::tower::BoxService< - super::metastore::CloseShardsRequest, - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, - delete_shards_svc: quickwit_common::tower::BoxService< - super::metastore::DeleteShardsRequest, - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, } impl Clone for ControlPlaneServiceTowerBlock { fn clone(&self) -> Self { @@ -487,8 +405,6 @@ impl Clone for ControlPlaneServiceTowerBlock { toggle_source_svc: self.toggle_source_svc.clone(), delete_source_svc: self.delete_source_svc.clone(), get_or_create_open_shards_svc: self.get_or_create_open_shards_svc.clone(), - close_shards_svc: self.close_shards_svc.clone(), - delete_shards_svc: self.delete_shards_svc.clone(), } } } @@ -532,18 +448,6 @@ impl ControlPlaneService for ControlPlaneServiceTowerBlock { ) -> crate::control_plane::ControlPlaneResult { self.get_or_create_open_shards_svc.ready().await?.call(request).await } - async fn close_shards( - &mut self, - request: super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.close_shards_svc.ready().await?.call(request).await - } - async fn delete_shards( - &mut self, - request: super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.delete_shards_svc.ready().await?.call(request).await - } } #[derive(Debug, Default)] pub struct ControlPlaneServiceTowerBlockBuilder { @@ -601,24 +505,6 @@ pub struct ControlPlaneServiceTowerBlockBuilder { crate::control_plane::ControlPlaneError, >, >, - #[allow(clippy::type_complexity)] - close_shards_layer: Option< - quickwit_common::tower::BoxLayer< - Box, - super::metastore::CloseShardsRequest, - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, - >, - #[allow(clippy::type_complexity)] - delete_shards_layer: Option< - quickwit_common::tower::BoxLayer< - Box, - super::metastore::DeleteShardsRequest, - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, - >, } impl ControlPlaneServiceTowerBlockBuilder { pub fn shared_layer(mut self, layer: L) -> Self @@ -672,22 +558,6 @@ impl ControlPlaneServiceTowerBlockBuilder { >::Future: Send + 'static, - L::Service: tower::Service< - super::metastore::CloseShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, - L::Service: tower::Service< - super::metastore::DeleteShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, { self .create_index_layer = Some( @@ -711,13 +581,8 @@ impl ControlPlaneServiceTowerBlockBuilder { ); self .get_or_create_open_shards_layer = Some( - quickwit_common::tower::BoxLayer::new(layer.clone()), - ); - self - .close_shards_layer = Some( - quickwit_common::tower::BoxLayer::new(layer.clone()), + quickwit_common::tower::BoxLayer::new(layer), ); - self.delete_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); self } pub fn create_index_layer(mut self, layer: L) -> Self @@ -813,36 +678,6 @@ impl ControlPlaneServiceTowerBlockBuilder { ); self } - pub fn close_shards_layer(mut self, layer: L) -> Self - where - L: tower::Layer> + Send + Sync + 'static, - L::Service: tower::Service< - super::metastore::CloseShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, - { - self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); - self - } - pub fn delete_shards_layer(mut self, layer: L) -> Self - where - L: tower::Layer> + Send + Sync + 'static, - L::Service: tower::Service< - super::metastore::DeleteShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, - { - self.delete_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); - self - } pub fn build(self, instance: T) -> ControlPlaneServiceClient where T: ControlPlaneService, @@ -912,16 +747,6 @@ impl ControlPlaneServiceTowerBlockBuilder { } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) }; - let close_shards_svc = if let Some(layer) = self.close_shards_layer { - layer.layer(boxed_instance.clone()) - } else { - quickwit_common::tower::BoxService::new(boxed_instance.clone()) - }; - let delete_shards_svc = if let Some(layer) = self.delete_shards_layer { - layer.layer(boxed_instance.clone()) - } else { - quickwit_common::tower::BoxService::new(boxed_instance.clone()) - }; let tower_block = ControlPlaneServiceTowerBlock { inner: boxed_instance.clone(), create_index_svc, @@ -930,8 +755,6 @@ impl ControlPlaneServiceTowerBlockBuilder { toggle_source_svc, delete_source_svc, get_or_create_open_shards_svc, - close_shards_svc, - delete_shards_svc, }; ControlPlaneServiceClient::new(tower_block) } @@ -1061,24 +884,6 @@ where GetOrCreateOpenShardsResponse, crate::control_plane::ControlPlaneError, >, - > - + tower::Service< - super::metastore::CloseShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - Future = BoxFuture< - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, - > - + tower::Service< - super::metastore::DeleteShardsRequest, - Response = super::metastore::EmptyResponse, - Error = crate::control_plane::ControlPlaneError, - Future = BoxFuture< - super::metastore::EmptyResponse, - crate::control_plane::ControlPlaneError, - >, >, { async fn create_index( @@ -1119,18 +924,6 @@ where ) -> crate::control_plane::ControlPlaneResult { self.call(request).await } - async fn close_shards( - &mut self, - request: super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await - } - async fn delete_shards( - &mut self, - request: super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await - } } #[derive(Debug, Clone)] pub struct ControlPlaneServiceGrpcClientAdapter { @@ -1228,26 +1021,6 @@ where .map(|response| response.into_inner()) .map_err(|error| error.into()) } - async fn close_shards( - &mut self, - request: super::metastore::CloseShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner - .close_shards(request) - .await - .map(|response| response.into_inner()) - .map_err(|error| error.into()) - } - async fn delete_shards( - &mut self, - request: super::metastore::DeleteShardsRequest, - ) -> crate::control_plane::ControlPlaneResult { - self.inner - .delete_shards(request) - .await - .map(|response| response.into_inner()) - .map_err(|error| error.into()) - } } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServerAdapter { @@ -1330,28 +1103,6 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } - async fn close_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - self.inner - .clone() - .close_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(|error| error.into()) - } - async fn delete_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - self.inner - .clone() - .delete_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(|error| error.into()) - } } /// Generated client implementations. pub mod control_plane_service_grpc_client { @@ -1631,70 +1382,6 @@ pub mod control_plane_service_grpc_client { ); self.inner.unary(req, path, codec).await } - /// Closes a list of shards. This RPC is a metastore callback. - pub async fn close_shards( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.control_plane.ControlPlaneService/CloseShards", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.control_plane.ControlPlaneService", - "CloseShards", - ), - ); - self.inner.unary(req, path, codec).await - } - /// Deletes a list of shards. This RPC is a metastore callback. - pub async fn delete_shards( - &mut self, - request: impl tonic::IntoRequest< - super::super::metastore::DeleteShardsRequest, - >, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.control_plane.ControlPlaneService/DeleteShards", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new( - "quickwit.control_plane.ControlPlaneService", - "DeleteShards", - ), - ); - self.inner.unary(req, path, codec).await - } } } /// Generated server implementations. @@ -1753,22 +1440,6 @@ pub mod control_plane_service_grpc_server { tonic::Response, tonic::Status, >; - /// Closes a list of shards. This RPC is a metastore callback. - async fn close_shards( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; - /// Deletes a list of shards. This RPC is a metastore callback. - async fn delete_shards( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServer { @@ -2141,104 +1812,6 @@ pub mod control_plane_service_grpc_server { }; Box::pin(fut) } - "/quickwit.control_plane.ControlPlaneService/CloseShards" => { - #[allow(non_camel_case_types)] - struct CloseShardsSvc(pub Arc); - impl< - T: ControlPlaneServiceGrpc, - > tonic::server::UnaryService< - super::super::metastore::CloseShardsRequest, - > for CloseShardsSvc { - type Response = super::super::metastore::EmptyResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; - fn call( - &mut self, - request: tonic::Request< - super::super::metastore::CloseShardsRequest, - >, - ) -> Self::Future { - let inner = Arc::clone(&self.0); - let fut = async move { - (*inner).close_shards(request).await - }; - Box::pin(fut) - } - } - let accept_compression_encodings = self.accept_compression_encodings; - let send_compression_encodings = self.send_compression_encodings; - let max_decoding_message_size = self.max_decoding_message_size; - let max_encoding_message_size = self.max_encoding_message_size; - let inner = self.inner.clone(); - let fut = async move { - let inner = inner.0; - let method = CloseShardsSvc(inner); - let codec = tonic::codec::ProstCodec::default(); - let mut grpc = tonic::server::Grpc::new(codec) - .apply_compression_config( - accept_compression_encodings, - send_compression_encodings, - ) - .apply_max_message_size_config( - max_decoding_message_size, - max_encoding_message_size, - ); - let res = grpc.unary(method, req).await; - Ok(res) - }; - Box::pin(fut) - } - "/quickwit.control_plane.ControlPlaneService/DeleteShards" => { - #[allow(non_camel_case_types)] - struct DeleteShardsSvc(pub Arc); - impl< - T: ControlPlaneServiceGrpc, - > tonic::server::UnaryService< - super::super::metastore::DeleteShardsRequest, - > for DeleteShardsSvc { - type Response = super::super::metastore::EmptyResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; - fn call( - &mut self, - request: tonic::Request< - super::super::metastore::DeleteShardsRequest, - >, - ) -> Self::Future { - let inner = Arc::clone(&self.0); - let fut = async move { - (*inner).delete_shards(request).await - }; - Box::pin(fut) - } - } - let accept_compression_encodings = self.accept_compression_encodings; - let send_compression_encodings = self.send_compression_encodings; - let max_decoding_message_size = self.max_decoding_message_size; - let max_encoding_message_size = self.max_encoding_message_size; - let inner = self.inner.clone(); - let fut = async move { - let inner = inner.0; - let method = DeleteShardsSvc(inner); - let codec = tonic::codec::ProstCodec::default(); - let mut grpc = tonic::server::Grpc::new(codec) - .apply_compression_config( - accept_compression_encodings, - send_compression_encodings, - ) - .apply_max_message_size_config( - max_decoding_message_size, - max_encoding_message_size, - ); - let res = grpc.unary(method, req).await; - Ok(res) - }; - Box::pin(fut) - } _ => { Box::pin(async move { Ok( 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 3e079a9aa20..6bedf10f60a 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -45,8 +45,8 @@ pub struct PersistSuccess { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, optional, tag = "4")] - pub replication_position_inclusive: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -140,8 +140,10 @@ pub struct ReplicateSubrequest { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, optional, tag = "4")] - pub from_position_exclusive: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub from_position_exclusive: ::core::option::Option, + #[prost(message, optional, tag = "5")] + pub to_position_inclusive: ::core::option::Option, #[prost(message, optional, tag = "6")] pub doc_batch: ::core::option::Option, } @@ -169,8 +171,8 @@ pub struct ReplicateSuccess { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, optional, tag = "4")] - pub replica_position_inclusive: ::core::option::Option, + #[prost(message, optional, tag = "4")] + pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -204,8 +206,8 @@ pub struct TruncateSubrequest { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, tag = "4")] - pub to_position_inclusive: u64, + #[prost(message, optional, tag = "4")] + pub to_position_inclusive: ::core::option::Option, } /// TODO #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] @@ -224,10 +226,10 @@ pub struct OpenFetchStreamRequest { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "4")] pub shard_id: u64, - #[prost(uint64, optional, tag = "5")] - pub from_position_exclusive: ::core::option::Option, - #[prost(uint64, optional, tag = "6")] - pub to_position_inclusive: ::core::option::Option, + #[prost(message, optional, tag = "5")] + pub from_position_exclusive: ::core::option::Option, + #[prost(message, optional, tag = "6")] + pub to_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -239,10 +241,12 @@ pub struct FetchResponseV2 { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, tag = "4")] - pub from_position_inclusive: u64, - #[prost(message, optional, tag = "5")] + #[prost(message, optional, tag = "4")] pub mrecord_batch: ::core::option::Option, + #[prost(message, optional, tag = "5")] + pub from_position_exclusive: ::core::option::Option, + #[prost(message, optional, tag = "6")] + pub to_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index 998532ad3e7..63d43d901d6 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -37,8 +37,9 @@ pub struct IngestSuccess { pub source_id: ::prost::alloc::string::String, #[prost(uint64, tag = "3")] pub shard_id: u64, - #[prost(uint64, tag = "4")] - pub replication_position: u64, + /// Replication position inclusive. + #[prost(message, optional, tag = "4")] + pub replication_position_inclusive: ::core::option::Option, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs index c5ab1253f21..0979e2a6ef3 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -40,18 +40,14 @@ pub struct Shard { /// Mutable fields #[prost(enumeration = "ShardState", tag = "8")] pub shard_state: i32, - /// Position up to which the follower has acknowledged replication of the records written in its log. - #[prost(uint64, optional, tag = "9")] - #[serde(default, skip_serializing_if = "Option::is_none")] - pub replication_position_inclusive: ::core::option::Option, /// Position up to which indexers have indexed and published the records stored in the shard. /// It is updated asynchronously in a best effort manner by the indexers and indicates the position up to which the log can be safely truncated. - #[prost(string, tag = "10")] - #[serde(default, skip_serializing_if = "String::is_empty")] - pub publish_position_inclusive: ::prost::alloc::string::String, + #[prost(message, optional, tag = "9")] + #[serde(default, skip_serializing_if = "Option::is_none")] + pub publish_position_inclusive: ::core::option::Option, /// A publish token that ensures only one indexer works on a given shard at a time. /// For instance, if an indexer goes rogue, eventually the control plane will detect it and assign the shard to another indexer, which will override the publish token. - #[prost(string, optional, tag = "11")] + #[prost(string, optional, tag = "10")] #[serde(default, skip_serializing_if = "Option::is_none")] pub publish_token: ::core::option::Option<::prost::alloc::string::String>, } @@ -93,9 +89,8 @@ impl CommitTypeV2 { pub enum ShardState { /// The shard is open and accepts write requests. Open = 0, - /// The shard is open and still accepts write requests, but should no longer be advertised to ingest routers. - /// It is waiting for its leader or follower to close it with its final replication position, after which write requests will be rejected. - Closing = 1, + /// The ingester hosting the shard is unavailable. + Unavailable = 1, /// The shard is closed and cannot be written to. /// It can be safely deleted if the publish position is superior or equal to the replication position. Closed = 2, @@ -108,7 +103,7 @@ impl ShardState { pub fn as_str_name(&self) -> &'static str { match self { ShardState::Open => "OPEN", - ShardState::Closing => "CLOSING", + ShardState::Unavailable => "UNAVAILABLE", ShardState::Closed => "CLOSED", } } @@ -116,7 +111,7 @@ impl ShardState { pub fn from_str_name(value: &str) -> ::core::option::Option { match value { "OPEN" => Some(Self::Open), - "CLOSING" => Some(Self::Closing), + "UNAVAILABLE" => Some(Self::Unavailable), "CLOSED" => Some(Self::Closed), _ => None, } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 9e43549e210..dff55917add 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -321,69 +321,6 @@ pub struct AcquireShardsSubresponse { #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct CloseShardsRequest { - #[prost(message, repeated, tag = "1")] - pub subrequests: ::prost::alloc::vec::Vec, -} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CloseShardsSubrequest { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, - #[prost(enumeration = "super::ingest::ShardState", tag = "4")] - pub shard_state: i32, - #[prost(uint64, optional, tag = "5")] - pub replication_position_inclusive: ::core::option::Option, -} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CloseShardsResponse { - #[prost(message, repeated, tag = "1")] - pub successes: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "2")] - pub failures: ::prost::alloc::vec::Vec, -} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CloseShardsSuccess { - #[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(string, tag = "4")] - pub leader_id: ::prost::alloc::string::String, - #[prost(string, optional, tag = "5")] - pub follower_id: ::core::option::Option<::prost::alloc::string::String>, - #[prost(string, tag = "6")] - pub publish_position_inclusive: ::prost::alloc::string::String, -} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] -pub struct CloseShardsFailure { - #[prost(string, tag = "1")] - pub index_uid: ::prost::alloc::string::String, - #[prost(string, tag = "2")] - pub source_id: ::prost::alloc::string::String, - #[prost(uint64, tag = "3")] - pub shard_id: u64, - #[prost(enumeration = "CloseShardsFailureKind", tag = "4")] - pub failure_kind: i32, - #[prost(string, tag = "5")] - pub failure_message: ::prost::alloc::string::String, -} -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteShardsRequest { #[prost(message, repeated, tag = "1")] pub subrequests: ::prost::alloc::vec::Vec, @@ -498,34 +435,6 @@ impl SourceType { } } } -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[serde(rename_all = "snake_case")] -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum CloseShardsFailureKind { - InvalidArgument = 0, - NotFound = 1, -} -impl CloseShardsFailureKind { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - CloseShardsFailureKind::InvalidArgument => "INVALID_ARGUMENT", - CloseShardsFailureKind::NotFound => "NOT_FOUND", - } - } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "INVALID_ARGUMENT" => Some(Self::InvalidArgument), - "NOT_FOUND" => Some(Self::NotFound), - _ => None, - } - } -} /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; use quickwit_common::metrics::{PrometheusLabels, OwnedPrometheusLabels}; @@ -635,11 +544,6 @@ impl PrometheusLabels<1> for AcquireShardsRequest { OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("acquire_shards")]) } } -impl PrometheusLabels<1> for CloseShardsRequest { - fn labels(&self) -> OwnedPrometheusLabels<1usize> { - OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("close_shards")]) - } -} impl PrometheusLabels<1> for DeleteShardsRequest { fn labels(&self) -> OwnedPrometheusLabels<1usize> { OwnedPrometheusLabels::new([std::borrow::Cow::Borrowed("delete_shards")]) @@ -754,10 +658,6 @@ pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync &mut self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult; - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> crate::metastore::MetastoreResult; async fn delete_shards( &mut self, request: DeleteShardsRequest, @@ -963,12 +863,6 @@ impl MetastoreService for MetastoreServiceClient { ) -> crate::metastore::MetastoreResult { self.inner.acquire_shards(request).await } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> crate::metastore::MetastoreResult { - self.inner.close_shards(request).await - } async fn delete_shards( &mut self, request: DeleteShardsRequest, @@ -1119,12 +1013,6 @@ pub mod metastore_service_mock { ) -> crate::metastore::MetastoreResult { self.inner.lock().await.acquire_shards(request).await } - async fn close_shards( - &mut self, - request: super::CloseShardsRequest, - ) -> crate::metastore::MetastoreResult { - self.inner.lock().await.close_shards(request).await - } async fn delete_shards( &mut self, request: super::DeleteShardsRequest, @@ -1476,22 +1364,6 @@ impl tower::Service for Box { Box::pin(fut) } } -impl tower::Service for Box { - type Response = CloseShardsResponse; - type Error = crate::metastore::MetastoreError; - type Future = BoxFuture; - fn poll_ready( - &mut self, - _cx: &mut std::task::Context<'_>, - ) -> std::task::Poll> { - std::task::Poll::Ready(Ok(())) - } - fn call(&mut self, request: CloseShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.close_shards(request).await }; - Box::pin(fut) - } -} impl tower::Service for Box { type Response = DeleteShardsResponse; type Error = crate::metastore::MetastoreError; @@ -1628,11 +1500,6 @@ struct MetastoreServiceTowerBlock { AcquireShardsResponse, crate::metastore::MetastoreError, >, - close_shards_svc: quickwit_common::tower::BoxService< - CloseShardsRequest, - CloseShardsResponse, - crate::metastore::MetastoreError, - >, delete_shards_svc: quickwit_common::tower::BoxService< DeleteShardsRequest, DeleteShardsResponse, @@ -1670,7 +1537,6 @@ impl Clone for MetastoreServiceTowerBlock { list_stale_splits_svc: self.list_stale_splits_svc.clone(), open_shards_svc: self.open_shards_svc.clone(), acquire_shards_svc: self.acquire_shards_svc.clone(), - close_shards_svc: self.close_shards_svc.clone(), delete_shards_svc: self.delete_shards_svc.clone(), list_shards_svc: self.list_shards_svc.clone(), } @@ -1798,12 +1664,6 @@ impl MetastoreService for MetastoreServiceTowerBlock { ) -> crate::metastore::MetastoreResult { self.acquire_shards_svc.ready().await?.call(request).await } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> crate::metastore::MetastoreResult { - self.close_shards_svc.ready().await?.call(request).await - } async fn delete_shards( &mut self, request: DeleteShardsRequest, @@ -2006,15 +1866,6 @@ pub struct MetastoreServiceTowerBlockBuilder { >, >, #[allow(clippy::type_complexity)] - close_shards_layer: Option< - quickwit_common::tower::BoxLayer< - Box, - CloseShardsRequest, - CloseShardsResponse, - crate::metastore::MetastoreError, - >, - >, - #[allow(clippy::type_complexity)] delete_shards_layer: Option< quickwit_common::tower::BoxLayer< Box, @@ -2165,12 +2016,6 @@ impl MetastoreServiceTowerBlockBuilder { Error = crate::metastore::MetastoreError, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, - L::Service: tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, - Error = crate::metastore::MetastoreError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, L::Service: tower::Service< DeleteShardsRequest, Response = DeleteShardsResponse, @@ -2264,10 +2109,6 @@ impl MetastoreServiceTowerBlockBuilder { .acquire_shards_layer = Some( quickwit_common::tower::BoxLayer::new(layer.clone()), ); - self - .close_shards_layer = Some( - quickwit_common::tower::BoxLayer::new(layer.clone()), - ); self .delete_shards_layer = Some( quickwit_common::tower::BoxLayer::new(layer.clone()), @@ -2567,19 +2408,6 @@ impl MetastoreServiceTowerBlockBuilder { self.acquire_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); self } - pub fn close_shards_layer(mut self, layer: L) -> Self - where - L: tower::Layer> + Send + Sync + 'static, - L::Service: tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, - Error = crate::metastore::MetastoreError, - > + Clone + Send + Sync + 'static, - >::Future: Send + 'static, - { - self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); - self - } pub fn delete_shards_layer(mut self, layer: L) -> Self where L: tower::Layer> + Send + Sync + 'static, @@ -2752,11 +2580,6 @@ impl MetastoreServiceTowerBlockBuilder { } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) }; - let close_shards_svc = if let Some(layer) = self.close_shards_layer { - layer.layer(boxed_instance.clone()) - } else { - quickwit_common::tower::BoxService::new(boxed_instance.clone()) - }; let delete_shards_svc = if let Some(layer) = self.delete_shards_layer { layer.layer(boxed_instance.clone()) } else { @@ -2789,7 +2612,6 @@ impl MetastoreServiceTowerBlockBuilder { list_stale_splits_svc, open_shards_svc, acquire_shards_svc, - close_shards_svc, delete_shards_svc, list_shards_svc, }; @@ -2997,12 +2819,6 @@ where Error = crate::metastore::MetastoreError, Future = BoxFuture, > - + tower::Service< - CloseShardsRequest, - Response = CloseShardsResponse, - Error = crate::metastore::MetastoreError, - Future = BoxFuture, - > + tower::Service< DeleteShardsRequest, Response = DeleteShardsResponse, @@ -3136,12 +2952,6 @@ where ) -> crate::metastore::MetastoreResult { self.call(request).await } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> crate::metastore::MetastoreResult { - self.call(request).await - } async fn delete_shards( &mut self, request: DeleteShardsRequest, @@ -3401,16 +3211,6 @@ where .map(|response| response.into_inner()) .map_err(|error| error.into()) } - async fn close_shards( - &mut self, - request: CloseShardsRequest, - ) -> crate::metastore::MetastoreResult { - self.inner - .close_shards(request) - .await - .map(|response| response.into_inner()) - .map_err(|error| error.into()) - } async fn delete_shards( &mut self, request: DeleteShardsRequest, @@ -3684,17 +3484,6 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } - async fn close_shards( - &self, - request: tonic::Request, - ) -> Result, tonic::Status> { - self.inner - .clone() - .close_shards(request.into_inner()) - .await - .map(tonic::Response::new) - .map_err(|error| error.into()) - } async fn delete_shards( &self, request: tonic::Request, @@ -4376,33 +4165,6 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } - pub async fn close_shards( - &mut self, - request: impl tonic::IntoRequest, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - > { - self.inner - .ready() - .await - .map_err(|e| { - tonic::Status::new( - tonic::Code::Unknown, - format!("Service was not ready: {}", e.into()), - ) - })?; - let codec = tonic::codec::ProstCodec::default(); - let path = http::uri::PathAndQuery::from_static( - "/quickwit.metastore.MetastoreService/CloseShards", - ); - let mut req = request.into_request(); - req.extensions_mut() - .insert( - GrpcMethod::new("quickwit.metastore.MetastoreService", "CloseShards"), - ); - self.inner.unary(req, path, codec).await - } pub async fn delete_shards( &mut self, request: impl tonic::IntoRequest, @@ -4600,13 +4362,6 @@ pub mod metastore_service_grpc_server { tonic::Response, tonic::Status, >; - async fn close_shards( - &self, - request: tonic::Request, - ) -> std::result::Result< - tonic::Response, - tonic::Status, - >; async fn delete_shards( &self, request: tonic::Request, @@ -5621,52 +5376,6 @@ pub mod metastore_service_grpc_server { }; Box::pin(fut) } - "/quickwit.metastore.MetastoreService/CloseShards" => { - #[allow(non_camel_case_types)] - struct CloseShardsSvc(pub Arc); - impl< - T: MetastoreServiceGrpc, - > tonic::server::UnaryService - for CloseShardsSvc { - type Response = super::CloseShardsResponse; - type Future = BoxFuture< - tonic::Response, - tonic::Status, - >; - fn call( - &mut self, - request: tonic::Request, - ) -> Self::Future { - let inner = Arc::clone(&self.0); - let fut = async move { - (*inner).close_shards(request).await - }; - Box::pin(fut) - } - } - let accept_compression_encodings = self.accept_compression_encodings; - let send_compression_encodings = self.send_compression_encodings; - let max_decoding_message_size = self.max_decoding_message_size; - let max_encoding_message_size = self.max_encoding_message_size; - let inner = self.inner.clone(); - let fut = async move { - let inner = inner.0; - let method = CloseShardsSvc(inner); - let codec = tonic::codec::ProstCodec::default(); - let mut grpc = tonic::server::Grpc::new(codec) - .apply_compression_config( - accept_compression_encodings, - send_compression_encodings, - ) - .apply_max_message_size_config( - max_decoding_message_size, - max_encoding_message_size, - ); - let res = grpc.unary(method, req).await; - Ok(res) - }; - Box::pin(fut) - } "/quickwit.metastore.MetastoreService/DeleteShards" => { #[allow(non_camel_case_types)] struct DeleteShardsSvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index cd1bec40261..2ed07cd10c2 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -24,7 +24,8 @@ use anyhow::anyhow; use quickwit_actors::AskError; use thiserror; -use crate::{IndexUid, ServiceError, ServiceErrorCode, SourceId}; +use crate::types::{IndexUid, SourceId}; +use crate::{ServiceError, ServiceErrorCode}; include!("../codegen/quickwit/quickwit.indexing.rs"); diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs index 03a2c89a732..24d3aa1410a 100644 --- a/quickwit/quickwit-proto/src/ingest/ingester.rs +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -17,7 +17,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use crate::types::{queue_id, QueueId}; +use crate::types::{queue_id, Position, QueueId}; include!("../codegen/quickwit/quickwit.ingest.ingester.rs"); @@ -35,20 +35,20 @@ impl FetchResponseV2 { 0 } } - - pub fn to_position_inclusive(&self) -> Option { - let Some(mrecord_batch) = &self.mrecord_batch else { - return None; - }; - let num_mrecords = mrecord_batch.num_mrecords() as u64; - Some(self.from_position_inclusive + num_mrecords - 1) - } } impl OpenFetchStreamRequest { pub fn queue_id(&self) -> QueueId { queue_id(&self.index_uid, &self.source_id, self.shard_id) } + + pub fn from_position_exclusive(&self) -> Position { + self.from_position_exclusive.clone().unwrap_or_default() + } + + pub fn to_position_inclusive(&self) -> Position { + self.to_position_inclusive.clone().unwrap_or_default() + } } impl PersistSubrequest { @@ -127,16 +127,20 @@ impl ReplicateSubrequest { queue_id(&self.index_uid, &self.source_id, self.shard_id) } - pub fn to_position_inclusive(&self) -> Option { - let Some(doc_batch) = &self.doc_batch else { - return self.from_position_exclusive; - }; - let num_docs = doc_batch.num_docs() as u64; + pub fn from_position_exclusive(&self) -> Position { + self.from_position_exclusive.clone().unwrap_or_default() + } - match self.from_position_exclusive { - Some(from_position_exclusive) => Some(from_position_exclusive + num_docs), - None => Some(num_docs - 1), - } + pub fn to_position_inclusive(&self) -> Position { + self.to_position_inclusive.clone().unwrap_or_default() + } +} + +impl ReplicateSuccess { + pub fn replication_position_inclusive(&self) -> Position { + self.replication_position_inclusive + .clone() + .unwrap_or_default() } } @@ -144,60 +148,8 @@ impl TruncateSubrequest { pub fn queue_id(&self) -> QueueId { queue_id(&self.index_uid, &self.source_id, self.shard_id) } -} -#[cfg(test)] -mod tests { - use bytes::Bytes; - - use super::*; - use crate::ingest::{DocBatchV2, MRecordBatch}; - - #[test] - fn test_fetch_response_to_position_inclusive() { - let mut response = FetchResponseV2 { - index_uid: "test-index".to_string(), - source_id: "test-source".to_string(), - shard_id: 0, - from_position_inclusive: 0, - mrecord_batch: None, - }; - assert_eq!(response.to_position_inclusive(), None); - - response.mrecord_batch = Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc"), - mrecord_lengths: vec![10], - }); - assert_eq!(response.to_position_inclusive(), Some(0)); - - response.mrecord_batch = Some(MRecordBatch { - mrecord_buffer: Bytes::from_static(b"\0\0test-doc\0\0test-doc"), - mrecord_lengths: vec![10, 10], - }); - assert_eq!(response.to_position_inclusive(), Some(1)); - } - - #[test] - fn test_replicate_subrequest_to_position_inclusive() { - let mut subrequest = ReplicateSubrequest { - index_uid: "test-index:0".to_string(), - source_id: "test-source".to_string(), - shard_id: 0, - from_position_exclusive: None, - doc_batch: None, - }; - assert_eq!(subrequest.to_position_inclusive(), None); - - subrequest.from_position_exclusive = Some(0); - assert_eq!(subrequest.to_position_inclusive(), Some(0)); - - subrequest.doc_batch = Some(DocBatchV2 { - doc_buffer: Bytes::from_static(b"test-doc"), - doc_lengths: vec![8], - }); - assert_eq!(subrequest.to_position_inclusive(), Some(1)); - - subrequest.from_position_exclusive = None; - assert_eq!(subrequest.to_position_inclusive(), Some(0)); + pub fn to_position_inclusive(&self) -> Position { + self.to_position_inclusive.clone().unwrap_or_default() } } diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs index e606a8bbeb2..d04b12380d4 100644 --- a/quickwit/quickwit-proto/src/ingest/mod.rs +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -19,9 +19,11 @@ use bytes::Bytes; +use self::ingester::FetchResponseV2; use super::types::{NodeId, ShardId, SourceId}; -use super::{IndexUid, ServiceError, ServiceErrorCode}; +use super::{ServiceError, ServiceErrorCode}; use crate::control_plane::ControlPlaneError; +use crate::types::{queue_id, IndexUid, Position}; pub mod ingester; pub mod router; @@ -133,6 +135,16 @@ 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 @@ -160,31 +172,23 @@ impl MRecordBatch { impl Shard { pub fn is_open(&self) -> bool { - self.shard_state() == ShardState::Open + self.shard_state().is_open() } - pub fn is_closing(&self) -> bool { - self.shard_state() == ShardState::Closing + pub fn is_unavailable(&self) -> bool { + self.shard_state().is_unavailable() } pub fn is_closed(&self) -> bool { - self.shard_state() == ShardState::Closed + self.shard_state().is_closed() } - pub fn is_deletable(&self) -> bool { - self.is_closed() && !self.has_unpublished_docs() - } - - pub fn is_indexable(&self) -> bool { - !self.is_closed() || self.has_unpublished_docs() - } - - pub fn has_unpublished_docs(&self) -> bool { - self.publish_position_inclusive.parse::().ok() < self.replication_position_inclusive + pub fn queue_id(&self) -> super::types::QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) } - pub fn queue_id(&self) -> super::types::QueueId { - super::types::queue_id(&self.index_uid, &self.source_id, self.shard_id) + pub fn publish_position_inclusive(&self) -> Position { + self.publish_position_inclusive.clone().unwrap_or_default() } } @@ -193,40 +197,11 @@ impl ShardState { *self == ShardState::Open } - pub fn is_closing(&self) -> bool { - *self == ShardState::Closing + pub fn is_unavailable(&self) -> bool { + *self == ShardState::Unavailable } pub fn is_closed(&self) -> bool { *self == ShardState::Closed } } - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_shard_as_unpublished_docs() { - let shard = Shard { - publish_position_inclusive: "".to_string(), - replication_position_inclusive: None, - ..Default::default() - }; - assert!(!shard.has_unpublished_docs()); - - let shard = Shard { - publish_position_inclusive: "".to_string(), - replication_position_inclusive: Some(0), - ..Default::default() - }; - assert!(shard.has_unpublished_docs()); - - let shard = Shard { - publish_position_inclusive: "0".to_string(), - replication_position_inclusive: Some(0), - ..Default::default() - }; - assert!(!shard.has_unpublished_docs()); - } -} diff --git a/quickwit/quickwit-proto/src/metastore/events.rs b/quickwit/quickwit-proto/src/metastore/events.rs index a36e51a129f..2ebad1ab6a3 100644 --- a/quickwit/quickwit-proto/src/metastore/events.rs +++ b/quickwit/quickwit-proto/src/metastore/events.rs @@ -17,11 +17,13 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +// use quickwit_common::pubsub::Event; + use quickwit_common::pubsub::Event; use super::{ - AddSourceRequest, CloseShardsRequest, CreateIndexRequest, DeleteIndexRequest, - DeleteShardsRequest, DeleteSourceRequest, SourceType, ToggleSourceRequest, + AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, DeleteSourceRequest, SourceType, + ToggleSourceRequest, }; use crate::{IndexUid, SourceId}; @@ -63,12 +65,6 @@ pub struct DeleteSourceEvent { pub source_id: SourceId, } -impl Event for AddSourceEvent {} -impl Event for DeleteIndexEvent {} -impl Event for DeleteSourceEvent {} -impl Event for ToggleSourceEvent {} -impl Event for CloseShardsRequest {} -impl Event for DeleteShardsRequest {} impl Event for AddSourceRequest {} impl Event for CreateIndexRequest {} impl Event for DeleteIndexRequest {} diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index 4b432db9a7d..9f6f74de56d 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -22,9 +22,8 @@ use std::fmt; use quickwit_common::retry::Retryable; use serde::{Deserialize, Serialize}; -use crate::{ - queue_id, IndexId, IndexUid, QueueId, ServiceError, ServiceErrorCode, SourceId, SplitId, -}; +use crate::types::{IndexId, QueueId, SourceId, SplitId}; +use crate::{IndexUid, ServiceError, ServiceErrorCode}; pub mod events; @@ -218,14 +217,14 @@ impl SourceType { } impl IndexMetadataRequest { - pub fn for_index_uid(index_uid: String) -> Self { + pub fn for_index_uid(index_uid: IndexUid) -> Self { Self { - index_uid: Some(index_uid), + index_uid: Some(index_uid.into()), index_id: None, } } - pub fn for_index_id(index_id: String) -> Self { + pub fn for_index_id(index_id: IndexId) -> Self { Self { index_uid: None, index_id: Some(index_id), @@ -275,18 +274,6 @@ impl ListDeleteTasksRequest { } } -impl CloseShardsSuccess { - pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) - } -} - -impl CloseShardsFailure { - pub fn queue_id(&self) -> QueueId { - queue_id(&self.index_uid, &self.source_id, self.shard_id) - } -} - pub mod serde_utils { use serde::{Deserialize, Serialize}; diff --git a/quickwit/quickwit-proto/src/types.rs b/quickwit/quickwit-proto/src/types/mod.rs similarity index 99% rename from quickwit/quickwit-proto/src/types.rs rename to quickwit/quickwit-proto/src/types/mod.rs index 72d11372a0b..a7050596934 100644 --- a/quickwit/quickwit-proto/src/types.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -27,6 +27,10 @@ use std::str::FromStr; use serde::{Deserialize, Serialize}; pub use ulid::Ulid; +mod position; + +pub use position::Position; + pub type IndexId = String; pub type SourceId = String; diff --git a/quickwit/quickwit-proto/src/types/position.rs b/quickwit/quickwit-proto/src/types/position.rs new file mode 100644 index 00000000000..c87e7440f49 --- /dev/null +++ b/quickwit/quickwit-proto/src/types/position.rs @@ -0,0 +1,327 @@ +// Copyright (C) 2023 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::fmt; + +use bytes::Bytes; +use bytestring::ByteString; +use prost::{self, DecodeError}; +use serde::{Deserialize, Serialize}; + +const BEGINNING: &str = ""; + +const EOF: &str = "~eof"; + +#[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Ord, PartialOrd)] +pub struct Offset(ByteString); + +impl Offset { + pub fn as_str(&self) -> &str { + &self.0 + } + + pub fn as_i64(&self) -> Option { + self.0.parse::().ok() + } + + pub fn as_u64(&self) -> Option { + self.0.parse::().ok() + } + + pub fn as_usize(&self) -> Option { + self.0.parse::().ok() + } +} + +impl fmt::Display for Offset { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}", &self.0) + } +} + +/// Marks a position within a specific partition/shard of a source. +/// +/// The nature of the position depends on the source. +/// Each source must encode it as a `String` in such a way that +/// the lexicographical order matches the natural order of the +/// position. +/// +/// For instance, for u64, a 20-left-padded decimal representation +/// can be used. Alternatively, a base64 representation of their +/// big-endian representation can be used. +/// +/// The empty string can be used to represent the beginning of the source, +/// if no position makes sense. It can be built via `Position::default()`. +#[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Ord, PartialOrd)] +pub enum Position { + #[default] + Beginning, + Offset(Offset), + 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 as_str(&self) -> &str { + match self { + Self::Beginning => BEGINNING, + Self::Offset(offset) => offset.as_str(), + Self::Eof => EOF, + } + } + + pub fn as_i64(&self) -> Option { + match self { + Self::Beginning => None, + Self::Offset(offset) => offset.as_i64(), + Self::Eof => None, + } + } + + pub fn as_u64(&self) -> Option { + match self { + Self::Beginning => None, + Self::Offset(offset) => offset.as_u64(), + Self::Eof => None, + } + } + + pub fn as_usize(&self) -> Option { + match self { + Self::Beginning => None, + Self::Offset(offset) => offset.as_usize(), + Self::Eof => None, + } + } +} + +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}")) + } +} + +impl From> for Position +where Position: From +{ + fn from(offset_opt: Option) -> Self { + match offset_opt { + Some(offset) => Self::from(offset), + None => Self::Beginning, + } + } +} + +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 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)) + } + } +} + +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 Serialize for Position { + fn serialize(&self, serializer: S) -> Result { + serializer.serialize_str(self.as_str()) + } +} + +impl<'de> Deserialize<'de> for Position { + fn deserialize>(deserializer: D) -> Result { + let string = String::deserialize(deserializer)?; + Ok(Self::from(string)) + } +} + +impl prost::Message for Position { + fn encode_raw(&self, buf: &mut B) + where B: prost::bytes::BufMut { + prost::encoding::bytes::encode(1u32, &self.as_bytes(), buf); + } + + fn merge_field( + &mut self, + tag: u32, + wire_type: prost::encoding::WireType, + buf: &mut B, + ctx: prost::encoding::DecodeContext, + ) -> ::core::result::Result<(), prost::DecodeError> + where + B: prost::bytes::Buf, + { + const STRUCT_NAME: &str = "Position"; + + match tag { + 1u32 => { + let mut value = Vec::new(); + prost::encoding::bytes::merge(wire_type, &mut value, buf, ctx).map_err( + |mut error| { + error.push(STRUCT_NAME, "position"); + error + }, + )?; + let byte_string = ByteString::try_from(value) + .map_err(|_| DecodeError::new("position is not valid UTF-8"))?; + *self = Self::from(byte_string); + Ok(()) + } + _ => prost::encoding::skip_field(wire_type, tag, buf, ctx), + } + } + + #[inline] + fn encoded_len(&self) -> usize { + prost::encoding::bytes::encoded_len(1u32, &self.as_bytes()) + } + + fn clear(&mut self) { + *self = Self::default(); + } +} + +#[cfg(test)] +mod tests { + use prost::Message; + + 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); + } + + #[test] + fn test_position_json_serde_roundtrip() { + let serialized = serde_json::to_string(&Position::Beginning).unwrap(); + assert_eq!(serialized, r#""""#); + let deserialized: Position = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, Position::Beginning); + + let serialized = serde_json::to_string(&Position::from(0u64)).unwrap(); + assert_eq!(serialized, r#""00000000000000000000""#); + let deserialized: Position = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, Position::from(0u64)); + + let serialized = serde_json::to_string(&Position::Eof).unwrap(); + assert_eq!(serialized, r#""~eof""#); + let deserialized: Position = serde_json::from_str(&serialized).unwrap(); + assert_eq!(deserialized, Position::Eof); + } + + #[test] + fn test_position_prost_serde_roundtrip() { + let mut buffer = Vec::new(); + Position::Beginning.encode(&mut buffer).unwrap(); + assert_eq!( + Position::decode(Bytes::from(buffer)).unwrap(), + Position::Beginning + ); + + let mut buffer = Vec::new(); + Position::from("0").encode(&mut buffer).unwrap(); + assert_eq!( + Position::decode(Bytes::from(buffer)).unwrap(), + Position::from("0") + ); + + let mut buffer = Vec::new(); + Position::from(0u64).encode(&mut buffer).unwrap(); + assert_eq!( + Position::decode(Bytes::from(buffer)).unwrap(), + Position::from(0u64) + ); + + let mut buffer = Vec::new(); + Position::Eof.encode(&mut buffer).unwrap(); + assert_eq!( + Position::decode(Bytes::from(buffer)).unwrap(), + Position::Eof + ); + } +} diff --git a/quickwit/quickwit-serve/src/ingest_metastore.rs b/quickwit/quickwit-serve/src/ingest_metastore.rs deleted file mode 100644 index 44323d52b26..00000000000 --- a/quickwit/quickwit-serve/src/ingest_metastore.rs +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright (C) 2023 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use async_trait::async_trait; -use quickwit_proto::ingest::{IngestV2Error, IngestV2Result}; -use quickwit_proto::metastore::{ - CloseShardsRequest, CloseShardsResponse, DeleteShardsRequest, DeleteShardsResponse, - MetastoreService, MetastoreServiceClient, -}; - -// TODO: Remove when the metastore is code generated in `quickwit-proto`. - -/// Implementation of the [`quickwit_ingest::IngestMetastore`] trait. See comment in the module -/// where it is defined for more details about why this is required. -#[derive(Clone)] -pub(crate) struct IngestMetastoreImpl { - metastore: MetastoreServiceClient, -} - -impl IngestMetastoreImpl { - pub fn new(metastore: MetastoreServiceClient) -> Self { - Self { metastore } - } -} - -#[async_trait] -impl quickwit_ingest::IngestMetastore for IngestMetastoreImpl { - async fn close_shards( - &self, - request: CloseShardsRequest, - ) -> IngestV2Result { - self.metastore - .clone() - .close_shards(request) - .await - .map_err(|error| IngestV2Error::Internal(error.to_string())) - } - - async fn delete_shards( - &self, - request: DeleteShardsRequest, - ) -> IngestV2Result { - self.metastore - .clone() - .delete_shards(request) - .await - .map_err(|error| IngestV2Error::Internal(error.to_string())) - } -} diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index e4f0ca767d7..c0a58fcafce 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -27,7 +27,6 @@ mod health_check_api; mod index_api; mod indexing_api; mod ingest_api; -mod ingest_metastore; mod json_api_response; mod metrics; mod metrics_api; @@ -52,7 +51,6 @@ use anyhow::Context; use byte_unit::n_mib_bytes; pub use format::BodyFormat; use futures::{Stream, StreamExt}; -use ingest_metastore::IngestMetastoreImpl; use itertools::Itertools; use quickwit_actors::{ActorExitStatus, Mailbox, Universe}; use quickwit_cluster::{start_cluster_service, Cluster, ClusterChange, ClusterMember}; @@ -65,7 +63,7 @@ use quickwit_common::tower::{ use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; use quickwit_control_plane::control_plane::ControlPlane; -use quickwit_control_plane::{ControlPlaneEventSubscriber, IndexerNodeInfo, IndexerPool}; +use quickwit_control_plane::{IndexerNodeInfo, IndexerPool}; use quickwit_index_management::{IndexService as IndexManager, IndexServiceError}; use quickwit_indexing::actors::IndexingService; use quickwit_indexing::start_indexing_service; @@ -84,8 +82,8 @@ use quickwit_proto::indexing::IndexingServiceClient; use quickwit_proto::ingest::ingester::IngesterServiceClient; use quickwit_proto::ingest::router::IngestRouterServiceClient; use quickwit_proto::metastore::{ - CloseShardsRequest, DeleteShardsRequest, EntityKind, ListIndexesMetadataRequest, - MetastoreError, MetastoreService, MetastoreServiceClient, + EntityKind, ListIndexesMetadataRequest, MetastoreError, MetastoreService, + MetastoreServiceClient, }; use quickwit_proto::search::ReportSplitsRequest; use quickwit_proto::NodeId; @@ -136,7 +134,6 @@ struct QuickwitServices { /// The control plane listens to metastore events. /// We must maintain a reference to the subscription handles to continue receiving /// notifications. Otherwise, the subscriptions are dropped. - _control_plane_event_subscription_handles_opt: Option, _report_splits_subscription_handle_opt: Option>, } @@ -325,13 +322,7 @@ pub async fn serve_quickwit( ) .await?; - // Setup control plane event subscriptions. - let control_plane_event_subscription_handles_opt = setup_control_plane_event_subscriptions( - &node_config, - &event_broker, - &control_plane_service, - ); - + // Set up the "control plane proxy" for the metastore. let metastore_through_control_plane = MetastoreServiceClient::new(ControlPlaneMetastore::new( control_plane_service.clone(), metastore_client.clone(), @@ -373,7 +364,6 @@ pub async fn serve_quickwit( let (ingest_router_service, ingester_service_opt) = setup_ingest_v2( &node_config, &cluster, - metastore_client.clone(), control_plane_service.clone(), ingester_pool, ) @@ -470,7 +460,6 @@ pub async fn serve_quickwit( metastore_server_opt, metastore_client: metastore_through_control_plane.clone(), control_plane_service, - _control_plane_event_subscription_handles_opt: control_plane_event_subscription_handles_opt, _report_splits_subscription_handle_opt: report_splits_subscription_handle_opt, index_manager, indexing_service_opt, @@ -554,39 +543,9 @@ pub async fn serve_quickwit( Ok(actor_exit_statuses) } -#[allow(dead_code)] -#[derive(Debug)] -struct ControlPlaneEventSubscriptionHandles { - close_shards_event_subscription_handle: EventSubscriptionHandle, - delete_shards_event_subscription_handle: EventSubscriptionHandle, -} - -fn setup_control_plane_event_subscriptions( - config: &NodeConfig, - event_broker: &EventBroker, - control_plane_service: &ControlPlaneServiceClient, -) -> Option { - if !config.is_service_enabled(QuickwitService::Metastore) { - return None; - } - let control_plane_event_subscriber = - ControlPlaneEventSubscriber::new(control_plane_service.clone()); - - let close_shards_event_subscription_handle = - event_broker.subscribe::(control_plane_event_subscriber.clone()); - let delete_shards_event_subscription_handle = - event_broker.subscribe::(control_plane_event_subscriber.clone()); - let control_plane_subscription_handles = ControlPlaneEventSubscriptionHandles { - close_shards_event_subscription_handle, - delete_shards_event_subscription_handle, - }; - Some(control_plane_subscription_handles) -} - async fn setup_ingest_v2( config: &NodeConfig, cluster: &Cluster, - metastore: MetastoreServiceClient, control_plane: ControlPlaneServiceClient, ingester_pool: IngesterPool, ) -> anyhow::Result<(IngestRouterServiceClient, Option)> { @@ -612,7 +571,6 @@ async fn setup_ingest_v2( let ingester = Ingester::try_new( self_node_id.clone(), - Arc::new(IngestMetastoreImpl::new(metastore)), ingester_pool.clone(), &wal_dir_path, replication_factor,